From 3b29004d2439c03a7d9bfdf7c2edd757d3d8c240 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 6 Jan 2016 10:43:03 -0800 Subject: [PATCH 001/110] [SPARK-7675][ML][PYSPARK] sparkml params type conversion From JIRA: Currently, PySpark wrappers for spark.ml Scala classes are brittle when accepting Param types. E.g., Normalizer's "p" param cannot be set to "2" (an integer); it must be set to "2.0" (a float). Fixing this is not trivial since there does not appear to be a natural place to insert the conversion before Python wrappers call Java's Params setter method. A possible fix will be to include a method "_checkType" to PySpark's Param class which checks the type, prints an error if needed, and converts types when relevant (e.g., int to float, or scipy matrix to array). The Java wrapper method which copies params to Scala can call this method when available. This fix instead checks the types at set time since I think failing sooner is better, but I can switch it around to check at copy time if that would be better. So far this only converts int to float and other conversions (like scipymatrix to array) are left for the future. Author: Holden Karau Closes #9581 from holdenk/SPARK-7675-PySpark-sparkml-Params-type-conversion. --- python/pyspark/ml/param/__init__.py | 22 ++- .../ml/param/_shared_params_code_gen.py | 63 ++++---- python/pyspark/ml/param/shared.py | 144 +++++++++--------- python/pyspark/ml/tests.py | 22 +++ 4 files changed, 148 insertions(+), 103 deletions(-) diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 35c9b776a3d5e..92ce96aa3c4df 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -32,12 +32,13 @@ class Param(object): .. versionadded:: 1.3.0 """ - def __init__(self, parent, name, doc): + def __init__(self, parent, name, doc, expectedType=None): if not isinstance(parent, Identifiable): raise TypeError("Parent must be an Identifiable but got type %s." % type(parent)) self.parent = parent.uid self.name = str(name) self.doc = str(doc) + self.expectedType = expectedType def __str__(self): return str(self.parent) + "__" + self.name @@ -247,7 +248,24 @@ def _set(self, **kwargs): Sets user-supplied params. """ for param, value in kwargs.items(): - self._paramMap[getattr(self, param)] = value + p = getattr(self, param) + if p.expectedType is None or type(value) == p.expectedType or value is None: + self._paramMap[getattr(self, param)] = value + else: + try: + # Try and do "safe" conversions that don't lose information + if p.expectedType == float: + self._paramMap[getattr(self, param)] = float(value) + # Python 3 unified long & int + elif p.expectedType == int and type(value).__name__ == 'long': + self._paramMap[getattr(self, param)] = value + else: + raise Exception( + "Provided type {0} incompatible with type {1} for param {2}" + .format(type(value), p.expectedType, p)) + except ValueError: + raise Exception(("Failed to convert {0} to type {1} for param {2}" + .format(type(value), p.expectedType, p))) return self def _setDefault(self, **kwargs): diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index 0528dc1e3a6b9..82855bc4c75ba 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -38,7 +38,7 @@ # python _shared_params_code_gen.py > shared.py -def _gen_param_header(name, doc, defaultValueStr): +def _gen_param_header(name, doc, defaultValueStr, expectedType): """ Generates the header part for shared variables @@ -51,22 +51,26 @@ def _gen_param_header(name, doc, defaultValueStr): """ # a placeholder to make it appear in the generated doc - $name = Param(Params._dummy(), "$name", "$doc") + $name = Param(Params._dummy(), "$name", "$doc", $expectedType) def __init__(self): super(Has$Name, self).__init__() #: param for $doc - self.$name = Param(self, "$name", "$doc")''' + self.$name = Param(self, "$name", "$doc", $expectedType)''' if defaultValueStr is not None: template += ''' self._setDefault($name=$defaultValueStr)''' Name = name[0].upper() + name[1:] + expectedTypeName = str(expectedType) + if expectedType is not None: + expectedTypeName = expectedType.__name__ return template \ .replace("$name", name) \ .replace("$Name", Name) \ .replace("$doc", doc) \ - .replace("$defaultValueStr", str(defaultValueStr)) + .replace("$defaultValueStr", str(defaultValueStr)) \ + .replace("$expectedType", expectedTypeName) def _gen_param_code(name, doc, defaultValueStr): @@ -84,7 +88,7 @@ def set$Name(self, value): """ Sets the value of :py:attr:`$name`. """ - self._paramMap[self.$name] = value + self._set($name=value) return self def get$Name(self): @@ -105,44 +109,45 @@ def get$Name(self): print("\n# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py.\n") print("from pyspark.ml.param import Param, Params\n\n") shared = [ - ("maxIter", "max number of iterations (>= 0).", None), - ("regParam", "regularization parameter (>= 0).", None), - ("featuresCol", "features column name.", "'features'"), - ("labelCol", "label column name.", "'label'"), - ("predictionCol", "prediction column name.", "'prediction'"), + ("maxIter", "max number of iterations (>= 0).", None, int), + ("regParam", "regularization parameter (>= 0).", None, float), + ("featuresCol", "features column name.", "'features'", str), + ("labelCol", "label column name.", "'label'", str), + ("predictionCol", "prediction column name.", "'prediction'", str), ("probabilityCol", "Column name for predicted class conditional probabilities. " + "Note: Not all models output well-calibrated probability estimates! These probabilities " + - "should be treated as confidences, not precise probabilities.", "'probability'"), - ("rawPredictionCol", "raw prediction (a.k.a. confidence) column name.", "'rawPrediction'"), - ("inputCol", "input column name.", None), - ("inputCols", "input column names.", None), - ("outputCol", "output column name.", "self.uid + '__output'"), - ("numFeatures", "number of features.", None), + "should be treated as confidences, not precise probabilities.", "'probability'", str), + ("rawPredictionCol", "raw prediction (a.k.a. confidence) column name.", "'rawPrediction'", + str), + ("inputCol", "input column name.", None, str), + ("inputCols", "input column names.", None, None), + ("outputCol", "output column name.", "self.uid + '__output'", str), + ("numFeatures", "number of features.", None, int), ("checkpointInterval", "set checkpoint interval (>= 1) or disable checkpoint (-1). " + - "E.g. 10 means that the cache will get checkpointed every 10 iterations.", None), - ("seed", "random seed.", "hash(type(self).__name__)"), - ("tol", "the convergence tolerance for iterative algorithms.", None), - ("stepSize", "Step size to be used for each iteration of optimization.", None), + "E.g. 10 means that the cache will get checkpointed every 10 iterations.", None, int), + ("seed", "random seed.", "hash(type(self).__name__)", int), + ("tol", "the convergence tolerance for iterative algorithms.", None, float), + ("stepSize", "Step size to be used for each iteration of optimization.", None, float), ("handleInvalid", "how to handle invalid entries. Options are skip (which will filter " + "out rows with bad values), or error (which will throw an errror). More options may be " + - "added later.", None), + "added later.", None, str), ("elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, " + - "the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.", "0.0"), - ("fitIntercept", "whether to fit an intercept term.", "True"), + "the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.", "0.0", float), + ("fitIntercept", "whether to fit an intercept term.", "True", bool), ("standardization", "whether to standardize the training features before fitting the " + - "model.", "True"), + "model.", "True", bool), ("thresholds", "Thresholds in multi-class classification to adjust the probability of " + "predicting each class. Array must have length equal to the number of classes, with " + "values >= 0. The class with largest value p/t is predicted, where p is the original " + - "probability of that class and t is the class' threshold.", None), + "probability of that class and t is the class' threshold.", None, None), ("weightCol", "weight column name. If this is not set or empty, we treat " + - "all instance weights as 1.0.", None), + "all instance weights as 1.0.", None, str), ("solver", "the solver algorithm for optimization. If this is not set or empty, " + - "default value is 'auto'.", "'auto'")] + "default value is 'auto'.", "'auto'", str)] code = [] - for name, doc, defaultValueStr in shared: - param_code = _gen_param_header(name, doc, defaultValueStr) + for name, doc, defaultValueStr, expectedType in shared: + param_code = _gen_param_header(name, doc, defaultValueStr, expectedType) code.append(param_code + "\n" + _gen_param_code(name, doc, defaultValueStr)) decisionTreeParams = [ diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 4d960801502c2..23f94314844f6 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -26,18 +26,18 @@ class HasMaxIter(Params): """ # a placeholder to make it appear in the generated doc - maxIter = Param(Params._dummy(), "maxIter", "max number of iterations (>= 0).") + maxIter = Param(Params._dummy(), "maxIter", "max number of iterations (>= 0).", int) def __init__(self): super(HasMaxIter, self).__init__() #: param for max number of iterations (>= 0). - self.maxIter = Param(self, "maxIter", "max number of iterations (>= 0).") + self.maxIter = Param(self, "maxIter", "max number of iterations (>= 0).", int) def setMaxIter(self, value): """ Sets the value of :py:attr:`maxIter`. """ - self._paramMap[self.maxIter] = value + self._set(maxIter=value) return self def getMaxIter(self): @@ -53,18 +53,18 @@ class HasRegParam(Params): """ # a placeholder to make it appear in the generated doc - regParam = Param(Params._dummy(), "regParam", "regularization parameter (>= 0).") + regParam = Param(Params._dummy(), "regParam", "regularization parameter (>= 0).", float) def __init__(self): super(HasRegParam, self).__init__() #: param for regularization parameter (>= 0). - self.regParam = Param(self, "regParam", "regularization parameter (>= 0).") + self.regParam = Param(self, "regParam", "regularization parameter (>= 0).", float) def setRegParam(self, value): """ Sets the value of :py:attr:`regParam`. """ - self._paramMap[self.regParam] = value + self._set(regParam=value) return self def getRegParam(self): @@ -80,19 +80,19 @@ class HasFeaturesCol(Params): """ # a placeholder to make it appear in the generated doc - featuresCol = Param(Params._dummy(), "featuresCol", "features column name.") + featuresCol = Param(Params._dummy(), "featuresCol", "features column name.", str) def __init__(self): super(HasFeaturesCol, self).__init__() #: param for features column name. - self.featuresCol = Param(self, "featuresCol", "features column name.") + self.featuresCol = Param(self, "featuresCol", "features column name.", str) self._setDefault(featuresCol='features') def setFeaturesCol(self, value): """ Sets the value of :py:attr:`featuresCol`. """ - self._paramMap[self.featuresCol] = value + self._set(featuresCol=value) return self def getFeaturesCol(self): @@ -108,19 +108,19 @@ class HasLabelCol(Params): """ # a placeholder to make it appear in the generated doc - labelCol = Param(Params._dummy(), "labelCol", "label column name.") + labelCol = Param(Params._dummy(), "labelCol", "label column name.", str) def __init__(self): super(HasLabelCol, self).__init__() #: param for label column name. - self.labelCol = Param(self, "labelCol", "label column name.") + self.labelCol = Param(self, "labelCol", "label column name.", str) self._setDefault(labelCol='label') def setLabelCol(self, value): """ Sets the value of :py:attr:`labelCol`. """ - self._paramMap[self.labelCol] = value + self._set(labelCol=value) return self def getLabelCol(self): @@ -136,19 +136,19 @@ class HasPredictionCol(Params): """ # a placeholder to make it appear in the generated doc - predictionCol = Param(Params._dummy(), "predictionCol", "prediction column name.") + predictionCol = Param(Params._dummy(), "predictionCol", "prediction column name.", str) def __init__(self): super(HasPredictionCol, self).__init__() #: param for prediction column name. - self.predictionCol = Param(self, "predictionCol", "prediction column name.") + self.predictionCol = Param(self, "predictionCol", "prediction column name.", str) self._setDefault(predictionCol='prediction') def setPredictionCol(self, value): """ Sets the value of :py:attr:`predictionCol`. """ - self._paramMap[self.predictionCol] = value + self._set(predictionCol=value) return self def getPredictionCol(self): @@ -164,19 +164,19 @@ class HasProbabilityCol(Params): """ # a placeholder to make it appear in the generated doc - probabilityCol = Param(Params._dummy(), "probabilityCol", "Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.") + probabilityCol = Param(Params._dummy(), "probabilityCol", "Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.", str) def __init__(self): super(HasProbabilityCol, self).__init__() #: param for Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities. - self.probabilityCol = Param(self, "probabilityCol", "Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.") + self.probabilityCol = Param(self, "probabilityCol", "Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.", str) self._setDefault(probabilityCol='probability') def setProbabilityCol(self, value): """ Sets the value of :py:attr:`probabilityCol`. """ - self._paramMap[self.probabilityCol] = value + self._set(probabilityCol=value) return self def getProbabilityCol(self): @@ -192,19 +192,19 @@ class HasRawPredictionCol(Params): """ # a placeholder to make it appear in the generated doc - rawPredictionCol = Param(Params._dummy(), "rawPredictionCol", "raw prediction (a.k.a. confidence) column name.") + rawPredictionCol = Param(Params._dummy(), "rawPredictionCol", "raw prediction (a.k.a. confidence) column name.", str) def __init__(self): super(HasRawPredictionCol, self).__init__() #: param for raw prediction (a.k.a. confidence) column name. - self.rawPredictionCol = Param(self, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name.") + self.rawPredictionCol = Param(self, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name.", str) self._setDefault(rawPredictionCol='rawPrediction') def setRawPredictionCol(self, value): """ Sets the value of :py:attr:`rawPredictionCol`. """ - self._paramMap[self.rawPredictionCol] = value + self._set(rawPredictionCol=value) return self def getRawPredictionCol(self): @@ -220,18 +220,18 @@ class HasInputCol(Params): """ # a placeholder to make it appear in the generated doc - inputCol = Param(Params._dummy(), "inputCol", "input column name.") + inputCol = Param(Params._dummy(), "inputCol", "input column name.", str) def __init__(self): super(HasInputCol, self).__init__() #: param for input column name. - self.inputCol = Param(self, "inputCol", "input column name.") + self.inputCol = Param(self, "inputCol", "input column name.", str) def setInputCol(self, value): """ Sets the value of :py:attr:`inputCol`. """ - self._paramMap[self.inputCol] = value + self._set(inputCol=value) return self def getInputCol(self): @@ -247,18 +247,18 @@ class HasInputCols(Params): """ # a placeholder to make it appear in the generated doc - inputCols = Param(Params._dummy(), "inputCols", "input column names.") + inputCols = Param(Params._dummy(), "inputCols", "input column names.", None) def __init__(self): super(HasInputCols, self).__init__() #: param for input column names. - self.inputCols = Param(self, "inputCols", "input column names.") + self.inputCols = Param(self, "inputCols", "input column names.", None) def setInputCols(self, value): """ Sets the value of :py:attr:`inputCols`. """ - self._paramMap[self.inputCols] = value + self._set(inputCols=value) return self def getInputCols(self): @@ -274,19 +274,19 @@ class HasOutputCol(Params): """ # a placeholder to make it appear in the generated doc - outputCol = Param(Params._dummy(), "outputCol", "output column name.") + outputCol = Param(Params._dummy(), "outputCol", "output column name.", str) def __init__(self): super(HasOutputCol, self).__init__() #: param for output column name. - self.outputCol = Param(self, "outputCol", "output column name.") + self.outputCol = Param(self, "outputCol", "output column name.", str) self._setDefault(outputCol=self.uid + '__output') def setOutputCol(self, value): """ Sets the value of :py:attr:`outputCol`. """ - self._paramMap[self.outputCol] = value + self._set(outputCol=value) return self def getOutputCol(self): @@ -302,18 +302,18 @@ class HasNumFeatures(Params): """ # a placeholder to make it appear in the generated doc - numFeatures = Param(Params._dummy(), "numFeatures", "number of features.") + numFeatures = Param(Params._dummy(), "numFeatures", "number of features.", int) def __init__(self): super(HasNumFeatures, self).__init__() #: param for number of features. - self.numFeatures = Param(self, "numFeatures", "number of features.") + self.numFeatures = Param(self, "numFeatures", "number of features.", int) def setNumFeatures(self, value): """ Sets the value of :py:attr:`numFeatures`. """ - self._paramMap[self.numFeatures] = value + self._set(numFeatures=value) return self def getNumFeatures(self): @@ -329,18 +329,18 @@ class HasCheckpointInterval(Params): """ # a placeholder to make it appear in the generated doc - checkpointInterval = Param(Params._dummy(), "checkpointInterval", "set checkpoint interval (>= 1) or disable checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations.") + checkpointInterval = Param(Params._dummy(), "checkpointInterval", "set checkpoint interval (>= 1) or disable checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations.", int) def __init__(self): super(HasCheckpointInterval, self).__init__() #: param for set checkpoint interval (>= 1) or disable checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations. - self.checkpointInterval = Param(self, "checkpointInterval", "set checkpoint interval (>= 1) or disable checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations.") + self.checkpointInterval = Param(self, "checkpointInterval", "set checkpoint interval (>= 1) or disable checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations.", int) def setCheckpointInterval(self, value): """ Sets the value of :py:attr:`checkpointInterval`. """ - self._paramMap[self.checkpointInterval] = value + self._set(checkpointInterval=value) return self def getCheckpointInterval(self): @@ -356,19 +356,19 @@ class HasSeed(Params): """ # a placeholder to make it appear in the generated doc - seed = Param(Params._dummy(), "seed", "random seed.") + seed = Param(Params._dummy(), "seed", "random seed.", int) def __init__(self): super(HasSeed, self).__init__() #: param for random seed. - self.seed = Param(self, "seed", "random seed.") + self.seed = Param(self, "seed", "random seed.", int) self._setDefault(seed=hash(type(self).__name__)) def setSeed(self, value): """ Sets the value of :py:attr:`seed`. """ - self._paramMap[self.seed] = value + self._set(seed=value) return self def getSeed(self): @@ -384,18 +384,18 @@ class HasTol(Params): """ # a placeholder to make it appear in the generated doc - tol = Param(Params._dummy(), "tol", "the convergence tolerance for iterative algorithms.") + tol = Param(Params._dummy(), "tol", "the convergence tolerance for iterative algorithms.", float) def __init__(self): super(HasTol, self).__init__() #: param for the convergence tolerance for iterative algorithms. - self.tol = Param(self, "tol", "the convergence tolerance for iterative algorithms.") + self.tol = Param(self, "tol", "the convergence tolerance for iterative algorithms.", float) def setTol(self, value): """ Sets the value of :py:attr:`tol`. """ - self._paramMap[self.tol] = value + self._set(tol=value) return self def getTol(self): @@ -411,18 +411,18 @@ class HasStepSize(Params): """ # a placeholder to make it appear in the generated doc - stepSize = Param(Params._dummy(), "stepSize", "Step size to be used for each iteration of optimization.") + stepSize = Param(Params._dummy(), "stepSize", "Step size to be used for each iteration of optimization.", float) def __init__(self): super(HasStepSize, self).__init__() #: param for Step size to be used for each iteration of optimization. - self.stepSize = Param(self, "stepSize", "Step size to be used for each iteration of optimization.") + self.stepSize = Param(self, "stepSize", "Step size to be used for each iteration of optimization.", float) def setStepSize(self, value): """ Sets the value of :py:attr:`stepSize`. """ - self._paramMap[self.stepSize] = value + self._set(stepSize=value) return self def getStepSize(self): @@ -438,18 +438,18 @@ class HasHandleInvalid(Params): """ # a placeholder to make it appear in the generated doc - handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later.") + handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later.", str) def __init__(self): super(HasHandleInvalid, self).__init__() #: param for how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later. - self.handleInvalid = Param(self, "handleInvalid", "how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later.") + self.handleInvalid = Param(self, "handleInvalid", "how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later.", str) def setHandleInvalid(self, value): """ Sets the value of :py:attr:`handleInvalid`. """ - self._paramMap[self.handleInvalid] = value + self._set(handleInvalid=value) return self def getHandleInvalid(self): @@ -465,19 +465,19 @@ class HasElasticNetParam(Params): """ # a placeholder to make it appear in the generated doc - elasticNetParam = Param(Params._dummy(), "elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.") + elasticNetParam = Param(Params._dummy(), "elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.", float) def __init__(self): super(HasElasticNetParam, self).__init__() #: param for the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty. - self.elasticNetParam = Param(self, "elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.") + self.elasticNetParam = Param(self, "elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.", float) self._setDefault(elasticNetParam=0.0) def setElasticNetParam(self, value): """ Sets the value of :py:attr:`elasticNetParam`. """ - self._paramMap[self.elasticNetParam] = value + self._set(elasticNetParam=value) return self def getElasticNetParam(self): @@ -493,19 +493,19 @@ class HasFitIntercept(Params): """ # a placeholder to make it appear in the generated doc - fitIntercept = Param(Params._dummy(), "fitIntercept", "whether to fit an intercept term.") + fitIntercept = Param(Params._dummy(), "fitIntercept", "whether to fit an intercept term.", bool) def __init__(self): super(HasFitIntercept, self).__init__() #: param for whether to fit an intercept term. - self.fitIntercept = Param(self, "fitIntercept", "whether to fit an intercept term.") + self.fitIntercept = Param(self, "fitIntercept", "whether to fit an intercept term.", bool) self._setDefault(fitIntercept=True) def setFitIntercept(self, value): """ Sets the value of :py:attr:`fitIntercept`. """ - self._paramMap[self.fitIntercept] = value + self._set(fitIntercept=value) return self def getFitIntercept(self): @@ -521,19 +521,19 @@ class HasStandardization(Params): """ # a placeholder to make it appear in the generated doc - standardization = Param(Params._dummy(), "standardization", "whether to standardize the training features before fitting the model.") + standardization = Param(Params._dummy(), "standardization", "whether to standardize the training features before fitting the model.", bool) def __init__(self): super(HasStandardization, self).__init__() #: param for whether to standardize the training features before fitting the model. - self.standardization = Param(self, "standardization", "whether to standardize the training features before fitting the model.") + self.standardization = Param(self, "standardization", "whether to standardize the training features before fitting the model.", bool) self._setDefault(standardization=True) def setStandardization(self, value): """ Sets the value of :py:attr:`standardization`. """ - self._paramMap[self.standardization] = value + self._set(standardization=value) return self def getStandardization(self): @@ -549,18 +549,18 @@ class HasThresholds(Params): """ # a placeholder to make it appear in the generated doc - thresholds = Param(Params._dummy(), "thresholds", "Thresholds in multi-class classification to adjust the probability of predicting each class. Array must have length equal to the number of classes, with values >= 0. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class' threshold.") + thresholds = Param(Params._dummy(), "thresholds", "Thresholds in multi-class classification to adjust the probability of predicting each class. Array must have length equal to the number of classes, with values >= 0. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class' threshold.", None) def __init__(self): super(HasThresholds, self).__init__() #: param for Thresholds in multi-class classification to adjust the probability of predicting each class. Array must have length equal to the number of classes, with values >= 0. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class' threshold. - self.thresholds = Param(self, "thresholds", "Thresholds in multi-class classification to adjust the probability of predicting each class. Array must have length equal to the number of classes, with values >= 0. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class' threshold.") + self.thresholds = Param(self, "thresholds", "Thresholds in multi-class classification to adjust the probability of predicting each class. Array must have length equal to the number of classes, with values >= 0. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class' threshold.", None) def setThresholds(self, value): """ Sets the value of :py:attr:`thresholds`. """ - self._paramMap[self.thresholds] = value + self._set(thresholds=value) return self def getThresholds(self): @@ -576,18 +576,18 @@ class HasWeightCol(Params): """ # a placeholder to make it appear in the generated doc - weightCol = Param(Params._dummy(), "weightCol", "weight column name. If this is not set or empty, we treat all instance weights as 1.0.") + weightCol = Param(Params._dummy(), "weightCol", "weight column name. If this is not set or empty, we treat all instance weights as 1.0.", str) def __init__(self): super(HasWeightCol, self).__init__() #: param for weight column name. If this is not set or empty, we treat all instance weights as 1.0. - self.weightCol = Param(self, "weightCol", "weight column name. If this is not set or empty, we treat all instance weights as 1.0.") + self.weightCol = Param(self, "weightCol", "weight column name. If this is not set or empty, we treat all instance weights as 1.0.", str) def setWeightCol(self, value): """ Sets the value of :py:attr:`weightCol`. """ - self._paramMap[self.weightCol] = value + self._set(weightCol=value) return self def getWeightCol(self): @@ -603,19 +603,19 @@ class HasSolver(Params): """ # a placeholder to make it appear in the generated doc - solver = Param(Params._dummy(), "solver", "the solver algorithm for optimization. If this is not set or empty, default value is 'auto'.") + solver = Param(Params._dummy(), "solver", "the solver algorithm for optimization. If this is not set or empty, default value is 'auto'.", str) def __init__(self): super(HasSolver, self).__init__() #: param for the solver algorithm for optimization. If this is not set or empty, default value is 'auto'. - self.solver = Param(self, "solver", "the solver algorithm for optimization. If this is not set or empty, default value is 'auto'.") + self.solver = Param(self, "solver", "the solver algorithm for optimization. If this is not set or empty, default value is 'auto'.", str) self._setDefault(solver='auto') def setSolver(self, value): """ Sets the value of :py:attr:`solver`. """ - self._paramMap[self.solver] = value + self._set(solver=value) return self def getSolver(self): @@ -658,7 +658,7 @@ def setMaxDepth(self, value): """ Sets the value of :py:attr:`maxDepth`. """ - self._paramMap[self.maxDepth] = value + self._set(maxDepth=value) return self def getMaxDepth(self): @@ -671,7 +671,7 @@ def setMaxBins(self, value): """ Sets the value of :py:attr:`maxBins`. """ - self._paramMap[self.maxBins] = value + self._set(maxBins=value) return self def getMaxBins(self): @@ -684,7 +684,7 @@ def setMinInstancesPerNode(self, value): """ Sets the value of :py:attr:`minInstancesPerNode`. """ - self._paramMap[self.minInstancesPerNode] = value + self._set(minInstancesPerNode=value) return self def getMinInstancesPerNode(self): @@ -697,7 +697,7 @@ def setMinInfoGain(self, value): """ Sets the value of :py:attr:`minInfoGain`. """ - self._paramMap[self.minInfoGain] = value + self._set(minInfoGain=value) return self def getMinInfoGain(self): @@ -710,7 +710,7 @@ def setMaxMemoryInMB(self, value): """ Sets the value of :py:attr:`maxMemoryInMB`. """ - self._paramMap[self.maxMemoryInMB] = value + self._set(maxMemoryInMB=value) return self def getMaxMemoryInMB(self): @@ -723,7 +723,7 @@ def setCacheNodeIds(self, value): """ Sets the value of :py:attr:`cacheNodeIds`. """ - self._paramMap[self.cacheNodeIds] = value + self._set(cacheNodeIds=value) return self def getCacheNodeIds(self): diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 7a16cf52cccb2..4eb17bfdcca90 100644 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -37,6 +37,7 @@ from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase from pyspark.sql import DataFrame, SQLContext, Row from pyspark.sql.functions import rand +from pyspark.ml.classification import LogisticRegression from pyspark.ml.evaluation import RegressionEvaluator from pyspark.ml.param import Param, Params from pyspark.ml.param.shared import HasMaxIter, HasInputCol, HasSeed @@ -92,6 +93,27 @@ class MockModel(MockTransformer, Model, HasFake): pass +class ParamTypeConversionTests(PySparkTestCase): + """ + Test that param type conversion happens. + """ + + def test_int_to_float(self): + from pyspark.mllib.linalg import Vectors + df = self.sc.parallelize([ + Row(label=1.0, weight=2.0, features=Vectors.dense(1.0))]).toDF() + lr = LogisticRegression(elasticNetParam=0) + lr.fit(df) + lr.setElasticNetParam(0) + lr.fit(df) + + def test_invalid_to_float(self): + from pyspark.mllib.linalg import Vectors + self.assertRaises(Exception, lambda: LogisticRegression(elasticNetParam="happy")) + lr = LogisticRegression(elasticNetParam=0) + self.assertRaises(Exception, lambda: lr.setElasticNetParam("panda")) + + class PipelineTests(PySparkTestCase): def test_pipeline(self): From 007da1a9dc3bb912da841cc0f5832a4fa28e6d9d Mon Sep 17 00:00:00 2001 From: Joshi Date: Wed, 6 Jan 2016 10:48:14 -0800 Subject: [PATCH 002/110] [SPARK-11531][ML] SparseVector error Msg PySpark SparseVector should have "Found duplicate indices" error message Author: Joshi Author: Rekha Joshi Closes #9525 from rekhajoshm/SPARK-11531. --- python/pyspark/mllib/linalg/__init__.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/pyspark/mllib/linalg/__init__.py b/python/pyspark/mllib/linalg/__init__.py index ae9ce58450905..131b855bf99c3 100644 --- a/python/pyspark/mllib/linalg/__init__.py +++ b/python/pyspark/mllib/linalg/__init__.py @@ -528,7 +528,9 @@ def __init__(self, size, *args): assert len(self.indices) == len(self.values), "index and value arrays not same length" for i in xrange(len(self.indices) - 1): if self.indices[i] >= self.indices[i + 1]: - raise TypeError("indices array must be sorted") + raise TypeError( + "Indices %s and %s are not strictly increasing" + % (self.indices[i], self.indices[i + 1])) def numNonzeros(self): """ From 95eb65163391b9e910277a948b72efccf6136e0c Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 6 Jan 2016 10:50:02 -0800 Subject: [PATCH 003/110] [SPARK-11945][ML][PYSPARK] Add computeCost to KMeansModel for PySpark spark.ml Add ```computeCost``` to ```KMeansModel``` as evaluator for PySpark spark.ml. Author: Yanbo Liang Closes #9931 from yanboliang/SPARK-11945. --- python/pyspark/ml/clustering.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 7bb8ab94e17df..9189c02220228 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -36,6 +36,14 @@ def clusterCenters(self): """Get the cluster centers, represented as a list of NumPy arrays.""" return [c.toArray() for c in self._call_java("clusterCenters")] + @since("2.0.0") + def computeCost(self, dataset): + """ + Return the K-means cost (sum of squared distances of points to their nearest center) + for this model on the given data. + """ + return self._call_java("computeCost", dataset) + @inherit_doc class KMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIter, HasTol, HasSeed): @@ -53,6 +61,8 @@ class KMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIter, HasTol >>> centers = model.clusterCenters() >>> len(centers) 2 + >>> model.computeCost(df) + 2.000... >>> transformed = model.transform(df).select("features", "prediction") >>> rows = transformed.collect() >>> rows[0].prediction == rows[1].prediction From 3aa3488225af12a77da3ba807906bc6a461ef11c Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 6 Jan 2016 10:52:25 -0800 Subject: [PATCH 004/110] [SPARK-11815][ML][PYSPARK] PySpark DecisionTreeClassifier & DecisionTreeRegressor should support setSeed PySpark ```DecisionTreeClassifier``` & ```DecisionTreeRegressor``` should support ```setSeed``` like what we do at Scala side. Author: Yanbo Liang Closes #9807 from yanboliang/spark-11815. --- python/pyspark/ml/classification.py | 13 ++++++++----- python/pyspark/ml/regression.py | 14 +++++++++----- 2 files changed, 17 insertions(+), 10 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 5599b8f3ecd88..265c6a14f1ca4 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -273,7 +273,7 @@ class GBTParams(TreeEnsembleParams): @inherit_doc class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasProbabilityCol, HasRawPredictionCol, DecisionTreeParams, - TreeClassifierParams, HasCheckpointInterval): + TreeClassifierParams, HasCheckpointInterval, HasSeed): """ `http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree` learning algorithm for classification. @@ -313,12 +313,14 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", probabilityCol="probability", rawPredictionCol="rawPrediction", maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini"): + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", + seed=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ probabilityCol="probability", rawPredictionCol="rawPrediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini") + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", \ + seed=None) """ super(DecisionTreeClassifier, self).__init__() self._java_obj = self._new_java_obj( @@ -335,12 +337,13 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre probabilityCol="probability", rawPredictionCol="rawPrediction", maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, - impurity="gini"): + impurity="gini", seed=None): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ probabilityCol="probability", rawPredictionCol="rawPrediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini") + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", \ + seed=None) Sets params for the DecisionTreeClassifier. """ kwargs = self.setParams._input_kwargs diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index a0bb8ceed8861..401bac0223ebd 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -386,7 +386,8 @@ class GBTParams(TreeEnsembleParams): @inherit_doc class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, - DecisionTreeParams, TreeRegressorParams, HasCheckpointInterval): + DecisionTreeParams, TreeRegressorParams, HasCheckpointInterval, + HasSeed): """ `http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree` learning algorithm for regression. @@ -415,11 +416,13 @@ class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance"): + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance", + seed=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance") + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, \ + impurity="variance", seed=None) """ super(DecisionTreeRegressor, self).__init__() self._java_obj = self._new_java_obj( @@ -435,11 +438,12 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, - impurity="variance"): + impurity="variance", seed=None): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance") + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, \ + impurity="variance", seed=None) Sets params for the DecisionTreeRegressor. """ kwargs = self.setParams._input_kwargs From ea489f14f11b2fdfb44c86634d2e2c2167b6ea18 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 6 Jan 2016 11:16:53 -0800 Subject: [PATCH 005/110] [SPARK-12573][SPARK-12574][SQL] Move SQL Parser from Hive to Catalyst This PR moves a major part of the new SQL parser to Catalyst. This is a prelude to start using this parser for all of our SQL parsing. The following key changes have been made: The ANTLR Parser & Supporting classes have been moved to the Catalyst project. They are now part of the ```org.apache.spark.sql.catalyst.parser``` package. These classes contained quite a bit of code that was originally from the Hive project, I have added aknowledgements whenever this applied. All Hive dependencies have been factored out. I have also taken this chance to clean-up the ```ASTNode``` class, and to improve the error handling. The HiveQl object that provides the functionality to convert an AST into a LogicalPlan has been refactored into three different classes, one for every SQL sub-project: - ```CatalystQl```: This implements Query and Expression parsing functionality. - ```SparkQl```: This is a subclass of CatalystQL and provides SQL/Core only functionality such as Explain and Describe. - ```HiveQl```: This is a subclass of ```SparkQl``` and this adds Hive-only functionality to the parser such as Analyze, Drop, Views, CTAS & Transforms. This class still depends on Hive. cc rxin Author: Herman van Hovell Closes #10583 from hvanhovell/SPARK-12575. --- dev/deps/spark-deps-hadoop-2.2 | 4 +- dev/deps/spark-deps-hadoop-2.3 | 4 +- dev/deps/spark-deps-hadoop-2.4 | 4 +- dev/deps/spark-deps-hadoop-2.6 | 4 +- pom.xml | 6 + project/SparkBuild.scala | 104 +- sql/catalyst/pom.xml | 22 + .../sql/catalyst}/parser/FromClauseParser.g | 4 +- .../sql/catalyst}/parser/IdentifiersParser.g | 4 +- .../sql/catalyst}/parser/SelectClauseParser.g | 4 +- .../sql/catalyst}/parser/SparkSqlLexer.g | 27 +- .../sql/catalyst}/parser/SparkSqlParser.g | 31 +- .../spark/sql/catalyst/parser/ParseUtils.java | 162 ++ .../spark/sql/catalyst/CatalystQl.scala | 961 ++++++++ .../spark/sql/catalyst/parser/ASTNode.scala | 93 + .../sql/catalyst/parser/ParseDriver.scala | 156 ++ .../sql/catalyst/parser/ParserConf.scala | 26 + .../scala/org/apache/spark/sql/SQLConf.scala | 20 +- .../apache/spark/sql/execution/SparkQl.scala | 84 + sql/hive/pom.xml | 20 - .../apache/spark/sql/parser/ASTErrorNode.java | 49 - .../org/apache/spark/sql/parser/ASTNode.java | 245 -- .../apache/spark/sql/parser/ParseDriver.java | 213 -- .../apache/spark/sql/parser/ParseError.java | 54 - .../spark/sql/parser/ParseException.java | 51 - .../apache/spark/sql/parser/ParseUtils.java | 96 - .../spark/sql/parser/SemanticAnalyzer.java | 406 ---- .../org/apache/spark/sql/hive/HiveQl.scala | 2050 ++++------------- .../spark/sql/hive/ErrorPositionSuite.scala | 9 +- 29 files changed, 2107 insertions(+), 2806 deletions(-) rename sql/{hive/src/main/antlr3/org/apache/spark/sql => catalyst/src/main/antlr3/org/apache/spark/sql/catalyst}/parser/FromClauseParser.g (98%) rename sql/{hive/src/main/antlr3/org/apache/spark/sql => catalyst/src/main/antlr3/org/apache/spark/sql/catalyst}/parser/IdentifiersParser.g (99%) rename sql/{hive/src/main/antlr3/org/apache/spark/sql => catalyst/src/main/antlr3/org/apache/spark/sql/catalyst}/parser/SelectClauseParser.g (97%) rename sql/{hive/src/main/antlr3/org/apache/spark/sql => catalyst/src/main/antlr3/org/apache/spark/sql/catalyst}/parser/SparkSqlLexer.g (93%) rename sql/{hive/src/main/antlr3/org/apache/spark/sql => catalyst/src/main/antlr3/org/apache/spark/sql/catalyst}/parser/SparkSqlParser.g (99%) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/parser/ParseUtils.java create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ASTNode.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserConf.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala delete mode 100644 sql/hive/src/main/java/org/apache/spark/sql/parser/ASTErrorNode.java delete mode 100644 sql/hive/src/main/java/org/apache/spark/sql/parser/ASTNode.java delete mode 100644 sql/hive/src/main/java/org/apache/spark/sql/parser/ParseDriver.java delete mode 100644 sql/hive/src/main/java/org/apache/spark/sql/parser/ParseError.java delete mode 100644 sql/hive/src/main/java/org/apache/spark/sql/parser/ParseException.java delete mode 100644 sql/hive/src/main/java/org/apache/spark/sql/parser/ParseUtils.java delete mode 100644 sql/hive/src/main/java/org/apache/spark/sql/parser/SemanticAnalyzer.java diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 44727f9876deb..e4373f79f7922 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -5,8 +5,7 @@ activation-1.1.jar akka-actor_2.10-2.3.11.jar akka-remote_2.10-2.3.11.jar akka-slf4j_2.10-2.3.11.jar -antlr-2.7.7.jar -antlr-runtime-3.4.jar +antlr-runtime-3.5.2.jar aopalliance-1.0.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar @@ -179,7 +178,6 @@ spire_2.10-0.7.4.jar stax-api-1.0-2.jar stax-api-1.0.1.jar stream-2.7.0.jar -stringtemplate-3.2.1.jar super-csv-2.2.0.jar tachyon-client-0.8.2.jar tachyon-underfs-hdfs-0.8.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 6014d50c6b6fd..7478181406d07 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -5,8 +5,7 @@ activation-1.1.1.jar akka-actor_2.10-2.3.11.jar akka-remote_2.10-2.3.11.jar akka-slf4j_2.10-2.3.11.jar -antlr-2.7.7.jar -antlr-runtime-3.4.jar +antlr-runtime-3.5.2.jar aopalliance-1.0.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar @@ -170,7 +169,6 @@ spire_2.10-0.7.4.jar stax-api-1.0-2.jar stax-api-1.0.1.jar stream-2.7.0.jar -stringtemplate-3.2.1.jar super-csv-2.2.0.jar tachyon-client-0.8.2.jar tachyon-underfs-hdfs-0.8.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index f56e6f4393e78..faffb8bf398a5 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -5,8 +5,7 @@ activation-1.1.1.jar akka-actor_2.10-2.3.11.jar akka-remote_2.10-2.3.11.jar akka-slf4j_2.10-2.3.11.jar -antlr-2.7.7.jar -antlr-runtime-3.4.jar +antlr-runtime-3.5.2.jar aopalliance-1.0.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar @@ -171,7 +170,6 @@ spire_2.10-0.7.4.jar stax-api-1.0-2.jar stax-api-1.0.1.jar stream-2.7.0.jar -stringtemplate-3.2.1.jar super-csv-2.2.0.jar tachyon-client-0.8.2.jar tachyon-underfs-hdfs-0.8.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index e37484473db2e..e703c7acd3876 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -5,8 +5,7 @@ activation-1.1.1.jar akka-actor_2.10-2.3.11.jar akka-remote_2.10-2.3.11.jar akka-slf4j_2.10-2.3.11.jar -antlr-2.7.7.jar -antlr-runtime-3.4.jar +antlr-runtime-3.5.2.jar aopalliance-1.0.jar apache-log4j-extras-1.2.17.jar apacheds-i18n-2.0.0-M15.jar @@ -177,7 +176,6 @@ spire_2.10-0.7.4.jar stax-api-1.0-2.jar stax-api-1.0.1.jar stream-2.7.0.jar -stringtemplate-3.2.1.jar super-csv-2.2.0.jar tachyon-client-0.8.2.jar tachyon-underfs-hdfs-0.8.2.jar diff --git a/pom.xml b/pom.xml index d0ac1eb39aabe..e414a8bfe6ce5 100644 --- a/pom.xml +++ b/pom.xml @@ -183,6 +183,7 @@ 3.5.2 1.3.9 0.9.2 + 3.5.2 ${java.home} @@ -1843,6 +1844,11 @@ + + org.antlr + antlr-runtime + ${antlr.version} + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index af1d36c6ea57b..5d4f19ab14a29 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -247,6 +247,9 @@ object SparkBuild extends PomBuild { /* Enable unidoc only for the root spark project */ enable(Unidoc.settings)(spark) + /* Catalyst ANTLR generation settings */ + enable(Catalyst.settings)(catalyst) + /* Spark SQL Core console settings */ enable(SQL.settings)(sql) @@ -357,6 +360,58 @@ object OldDeps { ) } +object Catalyst { + lazy val settings = Seq( + // ANTLR code-generation step. + // + // This has been heavily inspired by com.github.stefri.sbt-antlr (0.5.3). It fixes a number of + // build errors in the current plugin. + // Create Parser from ANTLR grammar files. + sourceGenerators in Compile += Def.task { + val log = streams.value.log + + val grammarFileNames = Seq( + "SparkSqlLexer.g", + "SparkSqlParser.g") + val sourceDir = (sourceDirectory in Compile).value / "antlr3" + val targetDir = (sourceManaged in Compile).value + + // Create default ANTLR Tool. + val antlr = new org.antlr.Tool + + // Setup input and output directories. + antlr.setInputDirectory(sourceDir.getPath) + antlr.setOutputDirectory(targetDir.getPath) + antlr.setForceRelativeOutput(true) + antlr.setMake(true) + + // Add grammar files. + grammarFileNames.flatMap(gFileName => (sourceDir ** gFileName).get).foreach { gFilePath => + val relGFilePath = (gFilePath relativeTo sourceDir).get.getPath + log.info("ANTLR: Grammar file '%s' detected.".format(relGFilePath)) + antlr.addGrammarFile(relGFilePath) + // We will set library directory multiple times here. However, only the + // last one has effect. Because the grammar files are located under the same directory, + // We assume there is only one library directory. + antlr.setLibDirectory(gFilePath.getParent) + } + + // Generate the parser. + antlr.process + if (antlr.getNumErrors > 0) { + log.error("ANTLR: Caught %d build errors.".format(antlr.getNumErrors)) + } + + // Return all generated java files. + (targetDir ** "*.java").get.toSeq + }.taskValue, + // Include ANTLR tokens files. + resourceGenerators in Compile += Def.task { + ((sourceManaged in Compile).value ** "*.tokens").get.toSeq + }.taskValue + ) +} + object SQL { lazy val settings = Seq( initialCommands in console := @@ -414,54 +469,7 @@ object Hive { // 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") }, - // ANTLR code-generation step. - // - // This has been heavily inspired by com.github.stefri.sbt-antlr (0.5.3). It fixes a number of - // build errors in the current plugin. - // Create Parser from ANTLR grammar files. - sourceGenerators in Compile += Def.task { - val log = streams.value.log - - val grammarFileNames = Seq( - "SparkSqlLexer.g", - "SparkSqlParser.g") - val sourceDir = (sourceDirectory in Compile).value / "antlr3" - val targetDir = (sourceManaged in Compile).value - - // Create default ANTLR Tool. - val antlr = new org.antlr.Tool - - // Setup input and output directories. - antlr.setInputDirectory(sourceDir.getPath) - antlr.setOutputDirectory(targetDir.getPath) - antlr.setForceRelativeOutput(true) - antlr.setMake(true) - - // Add grammar files. - grammarFileNames.flatMap(gFileName => (sourceDir ** gFileName).get).foreach { gFilePath => - val relGFilePath = (gFilePath relativeTo sourceDir).get.getPath - log.info("ANTLR: Grammar file '%s' detected.".format(relGFilePath)) - antlr.addGrammarFile(relGFilePath) - // We will set library directory multiple times here. However, only the - // last one has effect. Because the grammar files are located under the same directory, - // We assume there is only one library directory. - antlr.setLibDirectory(gFilePath.getParent) - } - - // Generate the parser. - antlr.process - if (antlr.getNumErrors > 0) { - log.error("ANTLR: Caught %d build errors.".format(antlr.getNumErrors)) - } - - // Return all generated java files. - (targetDir ** "*.java").get.toSeq - }.taskValue, - // Include ANTLR tokens files. - resourceGenerators in Compile += Def.task { - ((sourceManaged in Compile).value ** "*.tokens").get.toSeq - }.taskValue + fullClasspath in Test := (fullClasspath in Test).value.filterNot { f => f.toString.contains("jcl-over") } ) } diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index cfa520b7b9db2..76ca3f3bb1bfa 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -71,6 +71,10 @@ org.codehaus.janino janino + + org.antlr + antlr-runtime + target/scala-${scala.binary.version}/classes @@ -103,6 +107,24 @@ + + org.antlr + antlr3-maven-plugin + + + + antlr + + + + + ../catalyst/src/main/antlr3 + + **/SparkSqlLexer.g + **/SparkSqlParser.g + + + diff --git a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/FromClauseParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/FromClauseParser.g similarity index 98% rename from sql/hive/src/main/antlr3/org/apache/spark/sql/parser/FromClauseParser.g rename to sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/FromClauseParser.g index e4a80f0ce8ebf..ba6cfc60f045f 100644 --- a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/FromClauseParser.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/FromClauseParser.g @@ -13,6 +13,8 @@ WITHOUT 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 file is an adaptation of Hive's org/apache/hadoop/hive/ql/FromClauseParser.g grammar. */ parser grammar FromClauseParser; @@ -33,7 +35,7 @@ k=3; @Override public void displayRecognitionError(String[] tokenNames, RecognitionException e) { - gParent.errors.add(new ParseError(gParent, e, tokenNames)); + gParent.displayRecognitionError(tokenNames, e); } protected boolean useSQL11ReservedKeywordsForIdentifier() { return gParent.useSQL11ReservedKeywordsForIdentifier(); diff --git a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/IdentifiersParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g similarity index 99% rename from sql/hive/src/main/antlr3/org/apache/spark/sql/parser/IdentifiersParser.g rename to sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g index 9f1e168374f01..86c6bd610f912 100644 --- a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/IdentifiersParser.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g @@ -13,6 +13,8 @@ WITHOUT 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 file is an adaptation of Hive's org/apache/hadoop/hive/ql/IdentifiersParser.g grammar. */ parser grammar IdentifiersParser; @@ -33,7 +35,7 @@ k=3; @Override public void displayRecognitionError(String[] tokenNames, RecognitionException e) { - gParent.errors.add(new ParseError(gParent, e, tokenNames)); + gParent.displayRecognitionError(tokenNames, e); } protected boolean useSQL11ReservedKeywordsForIdentifier() { return gParent.useSQL11ReservedKeywordsForIdentifier(); diff --git a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SelectClauseParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g similarity index 97% rename from sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SelectClauseParser.g rename to sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g index 48bc8b0a300af..2d2bafb1ee34f 100644 --- a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SelectClauseParser.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g @@ -13,6 +13,8 @@ WITHOUT 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 file is an adaptation of Hive's org/apache/hadoop/hive/ql/SelectClauseParser.g grammar. */ parser grammar SelectClauseParser; @@ -33,7 +35,7 @@ k=3; @Override public void displayRecognitionError(String[] tokenNames, RecognitionException e) { - gParent.errors.add(new ParseError(gParent, e, tokenNames)); + gParent.displayRecognitionError(tokenNames, e); } protected boolean useSQL11ReservedKeywordsForIdentifier() { return gParent.useSQL11ReservedKeywordsForIdentifier(); diff --git a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlLexer.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlLexer.g similarity index 93% rename from sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlLexer.g rename to sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlLexer.g index ee1b8989b5aff..e01e7101d0b7e 100644 --- a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlLexer.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlLexer.g @@ -13,26 +13,37 @@ WITHOUT 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 file is an adaptation of Hive's org/apache/hadoop/hive/ql/HiveLexer.g grammar. */ lexer grammar SparkSqlLexer; @lexer::header { -package org.apache.spark.sql.parser; +package org.apache.spark.sql.catalyst.parser; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.conf.HiveConf; } @lexer::members { - private Configuration hiveConf; + private ParserConf parserConf; + private ParseErrorReporter reporter; - public void setHiveConf(Configuration hiveConf) { - this.hiveConf = hiveConf; + public void configure(ParserConf parserConf, ParseErrorReporter reporter) { + this.parserConf = parserConf; + this.reporter = reporter; } protected boolean allowQuotedId() { - String supportedQIds = HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_QUOTEDID_SUPPORT); - return !"none".equals(supportedQIds); + if (parserConf == null) { + return true; + } + return parserConf.supportQuotedId(); + } + + @Override + public void displayRecognitionError(String[] tokenNames, RecognitionException e) { + if (reporter != null) { + reporter.report(this, e, tokenNames); + } } } diff --git a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g similarity index 99% rename from sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlParser.g rename to sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g index 69574d713d0be..98b46794a630c 100644 --- a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlParser.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g @@ -13,6 +13,8 @@ WITHOUT 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 file is an adaptation of Hive's org/apache/hadoop/hive/ql/HiveParser.g grammar. */ parser grammar SparkSqlParser; @@ -369,18 +371,15 @@ TOK_SET_AUTOCOMMIT; // Package headers @header { -package org.apache.spark.sql.parser; +package org.apache.spark.sql.catalyst.parser; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.conf.HiveConf; } @members { - ArrayList errors = new ArrayList(); Stack msgs = new Stack(); private static HashMap xlateMap; @@ -563,9 +562,10 @@ import org.apache.hadoop.hive.conf.HiveConf; } @Override - public void displayRecognitionError(String[] tokenNames, - RecognitionException e) { - errors.add(new ParseError(this, e, tokenNames)); + public void displayRecognitionError(String[] tokenNames, RecognitionException e) { + if (reporter != null) { + reporter.report(this, e, tokenNames); + } } @Override @@ -654,15 +654,20 @@ import org.apache.hadoop.hive.conf.HiveConf; private CommonTree throwColumnNameException() throws RecognitionException { throw new FailedPredicateException(input, Arrays.toString(excludedCharForColumnName) + " can not be used in column name in create table statement.", ""); } - private Configuration hiveConf; - public void setHiveConf(Configuration hiveConf) { - this.hiveConf = hiveConf; + + private ParserConf parserConf; + private ParseErrorReporter reporter; + + public void configure(ParserConf parserConf, ParseErrorReporter reporter) { + this.parserConf = parserConf; + this.reporter = reporter; } + protected boolean useSQL11ReservedKeywordsForIdentifier() { - if(hiveConf==null){ - return false; + if (parserConf == null) { + return true; } - return !HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_SUPPORT_SQL11_RESERVED_KEYWORDS); + return !parserConf.supportSQL11ReservedKeywords(); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/parser/ParseUtils.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/parser/ParseUtils.java new file mode 100644 index 0000000000000..5bc87b680f9ad --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/parser/ParseUtils.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.sql.catalyst.parser; + +import java.io.UnsupportedEncodingException; + +/** + * A couple of utility methods that help with parsing ASTs. + * + * Both methods in this class were take from the SemanticAnalyzer in Hive: + * ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java + */ +public final class ParseUtils { + private ParseUtils() { + super(); + } + + public static String charSetString(String charSetName, String charSetString) + throws UnsupportedEncodingException { + // The character set name starts with a _, so strip that + charSetName = charSetName.substring(1); + if (charSetString.charAt(0) == '\'') { + return new String(unescapeSQLString(charSetString).getBytes(), charSetName); + } else // hex input is also supported + { + assert charSetString.charAt(0) == '0'; + assert charSetString.charAt(1) == 'x'; + charSetString = charSetString.substring(2); + + byte[] bArray = new byte[charSetString.length() / 2]; + int j = 0; + for (int i = 0; i < charSetString.length(); i += 2) { + int val = Character.digit(charSetString.charAt(i), 16) * 16 + + Character.digit(charSetString.charAt(i + 1), 16); + if (val > 127) { + val = val - 256; + } + bArray[j++] = (byte)val; + } + + return new String(bArray, charSetName); + } + } + + private static final int[] multiplier = new int[] {1000, 100, 10, 1}; + + @SuppressWarnings("nls") + public static String unescapeSQLString(String b) { + Character enclosure = null; + + // Some of the strings can be passed in as unicode. For example, the + // delimiter can be passed in as \002 - So, we first check if the + // string is a unicode number, else go back to the old behavior + StringBuilder sb = new StringBuilder(b.length()); + for (int i = 0; i < b.length(); i++) { + + char currentChar = b.charAt(i); + if (enclosure == null) { + if (currentChar == '\'' || b.charAt(i) == '\"') { + enclosure = currentChar; + } + // ignore all other chars outside the enclosure + continue; + } + + if (enclosure.equals(currentChar)) { + enclosure = null; + continue; + } + + if (currentChar == '\\' && (i + 6 < b.length()) && b.charAt(i + 1) == 'u') { + int code = 0; + int base = i + 2; + for (int j = 0; j < 4; j++) { + int digit = Character.digit(b.charAt(j + base), 16); + code += digit * multiplier[j]; + } + sb.append((char)code); + i += 5; + continue; + } + + if (currentChar == '\\' && (i + 4 < b.length())) { + char i1 = b.charAt(i + 1); + char i2 = b.charAt(i + 2); + char i3 = b.charAt(i + 3); + if ((i1 >= '0' && i1 <= '1') && (i2 >= '0' && i2 <= '7') + && (i3 >= '0' && i3 <= '7')) { + byte bVal = (byte) ((i3 - '0') + ((i2 - '0') * 8) + ((i1 - '0') * 8 * 8)); + byte[] bValArr = new byte[1]; + bValArr[0] = bVal; + String tmp = new String(bValArr); + sb.append(tmp); + i += 3; + continue; + } + } + + if (currentChar == '\\' && (i + 2 < b.length())) { + char n = b.charAt(i + 1); + switch (n) { + case '0': + sb.append("\0"); + break; + case '\'': + sb.append("'"); + break; + case '"': + sb.append("\""); + break; + case 'b': + sb.append("\b"); + break; + case 'n': + sb.append("\n"); + break; + case 'r': + sb.append("\r"); + break; + case 't': + sb.append("\t"); + break; + case 'Z': + sb.append("\u001A"); + break; + case '\\': + sb.append("\\"); + break; + // The following 2 lines are exactly what MySQL does TODO: why do we do this? + case '%': + sb.append("\\%"); + break; + case '_': + sb.append("\\_"); + break; + default: + sb.append(n); + } + i++; + } else { + sb.append(currentChar); + } + } + return sb.toString(); + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala new file mode 100644 index 0000000000000..42bdf25b61ea5 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala @@ -0,0 +1,961 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.sql.Date + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.Count +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.trees.CurrentOrigin +import org.apache.spark.sql.catalyst.parser._ +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.util.random.RandomSampler + +/** + * This class translates a HQL String to a Catalyst [[LogicalPlan]] or [[Expression]]. + */ +private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) { + object Token { + def unapply(node: ASTNode): Some[(String, List[ASTNode])] = { + CurrentOrigin.setPosition(node.line, node.positionInLine) + node.pattern + } + } + + + /** + * Returns the AST for the given SQL string. + */ + protected def getAst(sql: String): ASTNode = ParseDriver.parse(sql, conf) + + /** Creates LogicalPlan for a given HiveQL string. */ + def createPlan(sql: String): LogicalPlan = { + try { + createPlan(sql, ParseDriver.parse(sql, conf)) + } catch { + case e: MatchError => throw e + case e: AnalysisException => throw e + case e: Exception => + throw new AnalysisException(e.getMessage) + case e: NotImplementedError => + throw new AnalysisException( + s""" + |Unsupported language features in query: $sql + |${getAst(sql).treeString} + |$e + |${e.getStackTrace.head} + """.stripMargin) + } + } + + protected def createPlan(sql: String, tree: ASTNode): LogicalPlan = nodeToPlan(tree) + + def parseDdl(ddl: String): Seq[Attribute] = { + val tree = getAst(ddl) + assert(tree.text == "TOK_CREATETABLE", "Only CREATE TABLE supported.") + val tableOps = tree.children + val colList = tableOps + .find(_.text == "TOK_TABCOLLIST") + .getOrElse(sys.error("No columnList!")) + + colList.children.map(nodeToAttribute) + } + + protected def getClauses( + clauseNames: Seq[String], + nodeList: Seq[ASTNode]): Seq[Option[ASTNode]] = { + var remainingNodes = nodeList + val clauses = clauseNames.map { clauseName => + val (matches, nonMatches) = remainingNodes.partition(_.text.toUpperCase == clauseName) + remainingNodes = nonMatches ++ (if (matches.nonEmpty) matches.tail else Nil) + matches.headOption + } + + if (remainingNodes.nonEmpty) { + sys.error( + s"""Unhandled clauses: ${remainingNodes.map(_.treeString).mkString("\n")}. + |You are likely trying to use an unsupported Hive feature."""".stripMargin) + } + clauses + } + + protected def getClause(clauseName: String, nodeList: Seq[ASTNode]): ASTNode = + getClauseOption(clauseName, nodeList).getOrElse(sys.error( + s"Expected clause $clauseName missing from ${nodeList.map(_.treeString).mkString("\n")}")) + + protected def getClauseOption(clauseName: String, nodeList: Seq[ASTNode]): Option[ASTNode] = { + nodeList.filter { case ast: ASTNode => ast.text == clauseName } match { + case Seq(oneMatch) => Some(oneMatch) + case Seq() => None + case _ => sys.error(s"Found multiple instances of clause $clauseName") + } + } + + protected def nodeToAttribute(node: ASTNode): Attribute = node match { + case Token("TOK_TABCOL", Token(colName, Nil) :: dataType :: Nil) => + AttributeReference(colName, nodeToDataType(dataType), nullable = true)() + case _ => + noParseRule("Attribute", node) + } + + protected def nodeToDataType(node: ASTNode): DataType = node match { + case Token("TOK_DECIMAL", precision :: scale :: Nil) => + DecimalType(precision.text.toInt, scale.text.toInt) + case Token("TOK_DECIMAL", precision :: Nil) => + DecimalType(precision.text.toInt, 0) + case Token("TOK_DECIMAL", Nil) => DecimalType.USER_DEFAULT + case Token("TOK_BIGINT", Nil) => LongType + case Token("TOK_INT", Nil) => IntegerType + case Token("TOK_TINYINT", Nil) => ByteType + 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 + case Token("TOK_TIMESTAMP", Nil) => TimestampType + case Token("TOK_BINARY", Nil) => BinaryType + case Token("TOK_LIST", elementType :: Nil) => ArrayType(nodeToDataType(elementType)) + case Token("TOK_STRUCT", Token("TOK_TABCOLLIST", fields) :: Nil) => + StructType(fields.map(nodeToStructField)) + case Token("TOK_MAP", keyType :: valueType :: Nil) => + MapType(nodeToDataType(keyType), nodeToDataType(valueType)) + case _ => + noParseRule("DataType", node) + } + + protected def nodeToStructField(node: ASTNode): StructField = node match { + case Token("TOK_TABCOL", Token(fieldName, Nil) :: dataType :: Nil) => + StructField(fieldName, nodeToDataType(dataType), nullable = true) + case Token("TOK_TABCOL", Token(fieldName, Nil) :: dataType :: _ /* comment */:: Nil) => + StructField(fieldName, nodeToDataType(dataType), nullable = true) + case _ => + noParseRule("StructField", node) + } + + protected def extractTableIdent(tableNameParts: ASTNode): TableIdentifier = { + tableNameParts.children.map { + case Token(part, Nil) => cleanIdentifier(part) + } match { + case Seq(tableOnly) => TableIdentifier(tableOnly) + case Seq(databaseName, table) => TableIdentifier(table, Some(databaseName)) + case other => sys.error("Hive only supports tables names like 'tableName' " + + s"or 'databaseName.tableName', found '$other'") + } + } + + /** + * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) + * is equivalent to + * SELECT MAX(value) FROM src GROUP BY k1, k2 UNION SELECT MAX(value) FROM src GROUP BY k2 + * Check the following link for details. + * +https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C+Grouping+and+Rollup + * + * The bitmask denotes the grouping expressions validity for a grouping set, + * the bitmask also be called as grouping id (`GROUPING__ID`, the virtual column in Hive) + * e.g. In superset (k1, k2, k3), (bit 0: k1, bit 1: k2, and bit 2: k3), the grouping id of + * GROUPING SETS (k1, k2) and (k2) should be 3 and 2 respectively. + */ + protected def extractGroupingSet(children: Seq[ASTNode]): (Seq[Expression], Seq[Int]) = { + val (keyASTs, setASTs) = children.partition { + case Token("TOK_GROUPING_SETS_EXPRESSION", _) => false // grouping sets + case _ => true // grouping keys + } + + val keys = keyASTs.map(nodeToExpr) + val keyMap = keyASTs.zipWithIndex.toMap + + val bitmasks: Seq[Int] = setASTs.map { + case Token("TOK_GROUPING_SETS_EXPRESSION", null) => 0 + case Token("TOK_GROUPING_SETS_EXPRESSION", columns) => + columns.foldLeft(0)((bitmap, col) => { + val keyIndex = keyMap.find(_._1.treeEquals(col)).map(_._2) + bitmap | 1 << keyIndex.getOrElse( + throw new AnalysisException(s"${col.treeString} doesn't show up in the GROUP BY list")) + }) + case _ => sys.error("Expect GROUPING SETS clause") + } + + (keys, bitmasks) + } + + protected def nodeToPlan(node: ASTNode): LogicalPlan = node match { + case Token("TOK_QUERY", queryArgs @ Token("TOK_CTE" | "TOK_FROM" | "TOK_INSERT", _) :: _) => + val (fromClause: Option[ASTNode], insertClauses, cteRelations) = + queryArgs match { + case Token("TOK_CTE", ctes) :: Token("TOK_FROM", from) :: inserts => + val cteRelations = ctes.map { node => + val relation = nodeToRelation(node).asInstanceOf[Subquery] + relation.alias -> relation + } + (Some(from.head), inserts, Some(cteRelations.toMap)) + case Token("TOK_FROM", from) :: inserts => + (Some(from.head), inserts, None) + case Token("TOK_INSERT", _) :: Nil => + (None, queryArgs, None) + } + + // Return one query for each insert clause. + val queries = insertClauses.map { + case Token("TOK_INSERT", singleInsert) => + val ( + intoClause :: + destClause :: + selectClause :: + selectDistinctClause :: + whereClause :: + groupByClause :: + rollupGroupByClause :: + cubeGroupByClause :: + groupingSetsClause :: + orderByClause :: + havingClause :: + sortByClause :: + clusterByClause :: + distributeByClause :: + limitClause :: + lateralViewClause :: + windowClause :: Nil) = { + getClauses( + Seq( + "TOK_INSERT_INTO", + "TOK_DESTINATION", + "TOK_SELECT", + "TOK_SELECTDI", + "TOK_WHERE", + "TOK_GROUPBY", + "TOK_ROLLUP_GROUPBY", + "TOK_CUBE_GROUPBY", + "TOK_GROUPING_SETS", + "TOK_ORDERBY", + "TOK_HAVING", + "TOK_SORTBY", + "TOK_CLUSTERBY", + "TOK_DISTRIBUTEBY", + "TOK_LIMIT", + "TOK_LATERAL_VIEW", + "WINDOW"), + singleInsert) + } + + val relations = fromClause match { + case Some(f) => nodeToRelation(f) + case None => OneRowRelation + } + + val withWhere = whereClause.map { whereNode => + val Seq(whereExpr) = whereNode.children + Filter(nodeToExpr(whereExpr), relations) + }.getOrElse(relations) + + val select = (selectClause orElse selectDistinctClause) + .getOrElse(sys.error("No select clause.")) + + val transformation = nodeToTransformation(select.children.head, withWhere) + + val withLateralView = lateralViewClause.map { lv => + nodeToGenerate(lv.children.head, outer = false, withWhere) + }.getOrElse(withWhere) + + // The projection of the query can either be a normal projection, an aggregation + // (if there is a group by) or a script transformation. + val withProject: LogicalPlan = transformation.getOrElse { + val selectExpressions = + select.children.flatMap(selExprNodeToExpr).map(UnresolvedAlias(_)) + Seq( + groupByClause.map(e => e match { + case Token("TOK_GROUPBY", children) => + // Not a transformation so must be either project or aggregation. + Aggregate(children.map(nodeToExpr), selectExpressions, withLateralView) + case _ => sys.error("Expect GROUP BY") + }), + groupingSetsClause.map(e => e match { + case Token("TOK_GROUPING_SETS", children) => + val(groupByExprs, masks) = extractGroupingSet(children) + GroupingSets(masks, groupByExprs, withLateralView, selectExpressions) + case _ => sys.error("Expect GROUPING SETS") + }), + rollupGroupByClause.map(e => e match { + case Token("TOK_ROLLUP_GROUPBY", children) => + Aggregate( + Seq(Rollup(children.map(nodeToExpr))), + selectExpressions, + withLateralView) + case _ => sys.error("Expect WITH ROLLUP") + }), + cubeGroupByClause.map(e => e match { + case Token("TOK_CUBE_GROUPBY", children) => + Aggregate( + Seq(Cube(children.map(nodeToExpr))), + selectExpressions, + withLateralView) + case _ => sys.error("Expect WITH CUBE") + }), + Some(Project(selectExpressions, withLateralView))).flatten.head + } + + // Handle HAVING clause. + val withHaving = havingClause.map { h => + val havingExpr = h.children match { case Seq(hexpr) => nodeToExpr(hexpr) } + // Note that we added a cast to boolean. If the expression itself is already boolean, + // the optimizer will get rid of the unnecessary cast. + Filter(Cast(havingExpr, BooleanType), withProject) + }.getOrElse(withProject) + + // Handle SELECT DISTINCT + val withDistinct = + if (selectDistinctClause.isDefined) Distinct(withHaving) else withHaving + + // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause. + val withSort = + (orderByClause, sortByClause, distributeByClause, clusterByClause) match { + case (Some(totalOrdering), None, None, None) => + Sort(totalOrdering.children.map(nodeToSortOrder), global = true, withDistinct) + case (None, Some(perPartitionOrdering), None, None) => + Sort( + perPartitionOrdering.children.map(nodeToSortOrder), + global = false, withDistinct) + case (None, None, Some(partitionExprs), None) => + RepartitionByExpression( + partitionExprs.children.map(nodeToExpr), withDistinct) + case (None, Some(perPartitionOrdering), Some(partitionExprs), None) => + Sort( + perPartitionOrdering.children.map(nodeToSortOrder), global = false, + RepartitionByExpression( + partitionExprs.children.map(nodeToExpr), + withDistinct)) + case (None, None, None, Some(clusterExprs)) => + Sort( + clusterExprs.children.map(nodeToExpr).map(SortOrder(_, Ascending)), + global = false, + RepartitionByExpression( + clusterExprs.children.map(nodeToExpr), + withDistinct)) + case (None, None, None, None) => withDistinct + case _ => sys.error("Unsupported set of ordering / distribution clauses.") + } + + val withLimit = + limitClause.map(l => nodeToExpr(l.children.head)) + .map(Limit(_, withSort)) + .getOrElse(withSort) + + // Collect all window specifications defined in the WINDOW clause. + val windowDefinitions = windowClause.map(_.children.collect { + case Token("TOK_WINDOWDEF", + Token(windowName, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => + windowName -> nodesToWindowSpecification(spec) + }.toMap) + // Handle cases like + // window w1 as (partition by p_mfgr order by p_name + // range between 2 preceding and 2 following), + // w2 as w1 + val resolvedCrossReference = windowDefinitions.map { + windowDefMap => windowDefMap.map { + case (windowName, WindowSpecReference(other)) => + (windowName, windowDefMap(other).asInstanceOf[WindowSpecDefinition]) + case o => o.asInstanceOf[(String, WindowSpecDefinition)] + } + } + + val withWindowDefinitions = + resolvedCrossReference.map(WithWindowDefinition(_, withLimit)).getOrElse(withLimit) + + // TOK_INSERT_INTO means to add files to the table. + // TOK_DESTINATION means to overwrite the table. + val resultDestination = + (intoClause orElse destClause).getOrElse(sys.error("No destination found.")) + val overwrite = intoClause.isEmpty + nodeToDest( + resultDestination, + withWindowDefinitions, + overwrite) + } + + // If there are multiple INSERTS just UNION them together into on query. + val query = queries.reduceLeft(Union) + + // return With plan if there is CTE + cteRelations.map(With(query, _)).getOrElse(query) + + // HIVE-9039 renamed TOK_UNION => TOK_UNIONALL while adding TOK_UNIONDISTINCT + case Token("TOK_UNIONALL", left :: right :: Nil) => + Union(nodeToPlan(left), nodeToPlan(right)) + + case _ => + noParseRule("Plan", node) + } + + val allJoinTokens = "(TOK_.*JOIN)".r + val laterViewToken = "TOK_LATERAL_VIEW(.*)".r + protected def nodeToRelation(node: ASTNode): LogicalPlan = { + node match { + case Token("TOK_SUBQUERY", query :: Token(alias, Nil) :: Nil) => + Subquery(cleanIdentifier(alias), nodeToPlan(query)) + + case Token(laterViewToken(isOuter), selectClause :: relationClause :: Nil) => + nodeToGenerate( + selectClause, + outer = isOuter.nonEmpty, + nodeToRelation(relationClause)) + + /* All relations, possibly with aliases or sampling clauses. */ + case Token("TOK_TABREF", clauses) => + // If the last clause is not a token then it's the alias of the table. + val (nonAliasClauses, aliasClause) = + if (clauses.last.text.startsWith("TOK")) { + (clauses, None) + } else { + (clauses.dropRight(1), Some(clauses.last)) + } + + val (Some(tableNameParts) :: + splitSampleClause :: + bucketSampleClause :: Nil) = { + getClauses(Seq("TOK_TABNAME", "TOK_TABLESPLITSAMPLE", "TOK_TABLEBUCKETSAMPLE"), + nonAliasClauses) + } + + val tableIdent = extractTableIdent(tableNameParts) + val alias = aliasClause.map { case Token(a, Nil) => cleanIdentifier(a) } + val relation = UnresolvedRelation(tableIdent, alias) + + // Apply sampling if requested. + (bucketSampleClause orElse splitSampleClause).map { + case Token("TOK_TABLESPLITSAMPLE", + Token("TOK_ROWCOUNT", Nil) :: Token(count, Nil) :: Nil) => + Limit(Literal(count.toInt), relation) + case Token("TOK_TABLESPLITSAMPLE", + Token("TOK_PERCENT", Nil) :: Token(fraction, Nil) :: Nil) => + // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling + // function takes X PERCENT as the input and the range of X is [0, 100], we need to + // adjust the fraction. + require( + fraction.toDouble >= (0.0 - RandomSampler.roundingEpsilon) + && fraction.toDouble <= (100.0 + RandomSampler.roundingEpsilon), + s"Sampling fraction ($fraction) must be on interval [0, 100]") + Sample(0.0, fraction.toDouble / 100, withReplacement = false, + (math.random * 1000).toInt, + relation) + case Token("TOK_TABLEBUCKETSAMPLE", + Token(numerator, Nil) :: + Token(denominator, Nil) :: Nil) => + val fraction = numerator.toDouble / denominator.toDouble + Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, relation) + case a => + noParseRule("Sampling", a) + }.getOrElse(relation) + + case Token(allJoinTokens(joinToken), relation1 :: relation2 :: other) => + if (!(other.size <= 1)) { + sys.error(s"Unsupported join operation: $other") + } + + val joinType = joinToken match { + case "TOK_JOIN" => Inner + case "TOK_CROSSJOIN" => Inner + case "TOK_RIGHTOUTERJOIN" => RightOuter + case "TOK_LEFTOUTERJOIN" => LeftOuter + case "TOK_FULLOUTERJOIN" => FullOuter + case "TOK_LEFTSEMIJOIN" => LeftSemi + case "TOK_UNIQUEJOIN" => noParseRule("Unique Join", node) + case "TOK_ANTIJOIN" => noParseRule("Anti Join", node) + } + Join(nodeToRelation(relation1), + nodeToRelation(relation2), + joinType, + other.headOption.map(nodeToExpr)) + + case _ => + noParseRule("Relation", node) + } + } + + protected def nodeToSortOrder(node: ASTNode): SortOrder = node match { + case Token("TOK_TABSORTCOLNAMEASC", sortExpr :: Nil) => + SortOrder(nodeToExpr(sortExpr), Ascending) + case Token("TOK_TABSORTCOLNAMEDESC", sortExpr :: Nil) => + SortOrder(nodeToExpr(sortExpr), Descending) + case _ => + noParseRule("SortOrder", node) + } + + val destinationToken = "TOK_DESTINATION|TOK_INSERT_INTO".r + protected def nodeToDest( + node: ASTNode, + query: LogicalPlan, + overwrite: Boolean): LogicalPlan = node match { + case Token(destinationToken(), + Token("TOK_DIR", + Token("TOK_TMP_FILE", Nil) :: Nil) :: Nil) => + query + + case Token(destinationToken(), + Token("TOK_TAB", + tableArgs) :: Nil) => + val Some(tableNameParts) :: partitionClause :: Nil = + getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) + + val tableIdent = extractTableIdent(tableNameParts) + + val partitionKeys = partitionClause.map(_.children.map { + // Parse partitions. We also make keys case insensitive. + case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> Some(unquoteString(value)) + case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> None + }.toMap).getOrElse(Map.empty) + + InsertIntoTable( + UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, ifNotExists = false) + + case Token(destinationToken(), + Token("TOK_TAB", + tableArgs) :: + Token("TOK_IFNOTEXISTS", + ifNotExists) :: Nil) => + val Some(tableNameParts) :: partitionClause :: Nil = + getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) + + val tableIdent = extractTableIdent(tableNameParts) + + val partitionKeys = partitionClause.map(_.children.map { + // Parse partitions. We also make keys case insensitive. + case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> Some(unquoteString(value)) + case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> None + }.toMap).getOrElse(Map.empty) + + InsertIntoTable( + UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, ifNotExists = true) + + case _ => + noParseRule("Destination", node) + } + + protected def selExprNodeToExpr(node: ASTNode): Option[Expression] = node match { + case Token("TOK_SELEXPR", e :: Nil) => + Some(nodeToExpr(e)) + + case Token("TOK_SELEXPR", e :: Token(alias, Nil) :: Nil) => + Some(Alias(nodeToExpr(e), cleanIdentifier(alias))()) + + case Token("TOK_SELEXPR", e :: aliasChildren) => + val aliasNames = aliasChildren.collect { + case Token(name, Nil) => cleanIdentifier(name) + } + Some(MultiAlias(nodeToExpr(e), aliasNames)) + + /* Hints are ignored */ + case Token("TOK_HINTLIST", _) => None + + case _ => + noParseRule("Select", node) + } + + protected val escapedIdentifier = "`([^`]+)`".r + protected val doubleQuotedString = "\"([^\"]+)\"".r + protected val singleQuotedString = "'([^']+)'".r + + protected def unquoteString(str: String) = str match { + case singleQuotedString(s) => s + case doubleQuotedString(s) => s + case other => other + } + + /** Strips backticks from ident if present */ + protected def cleanIdentifier(ident: String): String = ident match { + case escapedIdentifier(i) => i + case plainIdent => plainIdent + } + + val numericAstTypes = Seq( + SparkSqlParser.Number, + SparkSqlParser.TinyintLiteral, + SparkSqlParser.SmallintLiteral, + SparkSqlParser.BigintLiteral, + SparkSqlParser.DecimalLiteral) + + /* Case insensitive matches */ + val COUNT = "(?i)COUNT".r + val SUM = "(?i)SUM".r + val AND = "(?i)AND".r + val OR = "(?i)OR".r + val NOT = "(?i)NOT".r + val TRUE = "(?i)TRUE".r + val FALSE = "(?i)FALSE".r + val LIKE = "(?i)LIKE".r + val RLIKE = "(?i)RLIKE".r + val REGEXP = "(?i)REGEXP".r + val IN = "(?i)IN".r + val DIV = "(?i)DIV".r + val BETWEEN = "(?i)BETWEEN".r + val WHEN = "(?i)WHEN".r + val CASE = "(?i)CASE".r + + protected def nodeToExpr(node: ASTNode): Expression = node match { + /* Attribute References */ + case Token("TOK_TABLE_OR_COL", Token(name, Nil) :: Nil) => + UnresolvedAttribute.quoted(cleanIdentifier(name)) + case Token(".", qualifier :: Token(attr, Nil) :: Nil) => + nodeToExpr(qualifier) match { + case UnresolvedAttribute(nameParts) => + UnresolvedAttribute(nameParts :+ cleanIdentifier(attr)) + case other => UnresolvedExtractValue(other, Literal(attr)) + } + + /* Stars (*) */ + case Token("TOK_ALLCOLREF", Nil) => UnresolvedStar(None) + // The format of dbName.tableName.* cannot be parsed by HiveParser. TOK_TABNAME will only + // has a single child which is tableName. + case Token("TOK_ALLCOLREF", Token("TOK_TABNAME", Token(name, Nil) :: Nil) :: Nil) => + UnresolvedStar(Some(UnresolvedAttribute.parseAttributeName(name))) + + /* Aggregate Functions */ + case Token("TOK_FUNCTIONDI", Token(COUNT(), Nil) :: args) => + Count(args.map(nodeToExpr)).toAggregateExpression(isDistinct = true) + case Token("TOK_FUNCTIONSTAR", Token(COUNT(), Nil) :: Nil) => + Count(Literal(1)).toAggregateExpression() + + /* Casts */ + case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), StringType) + 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) + case Token("TOK_FUNCTION", Token("TOK_BIGINT", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), LongType) + case Token("TOK_FUNCTION", Token("TOK_FLOAT", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), FloatType) + case Token("TOK_FUNCTION", Token("TOK_DOUBLE", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), DoubleType) + case Token("TOK_FUNCTION", Token("TOK_SMALLINT", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), ShortType) + case Token("TOK_FUNCTION", Token("TOK_TINYINT", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), ByteType) + case Token("TOK_FUNCTION", Token("TOK_BINARY", Nil) :: arg :: Nil) => + 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.text.toInt, scale.text.toInt)) + case Token("TOK_FUNCTION", Token("TOK_DECIMAL", precision :: Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), DecimalType(precision.text.toInt, 0)) + case Token("TOK_FUNCTION", Token("TOK_DECIMAL", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), DecimalType.USER_DEFAULT) + 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) => nodeToExpr(child) + 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)) + case Token("/", 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)) + 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)) + + /* 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)) + case Token(">=", left :: right:: Nil) => GreaterThanOrEqual(nodeToExpr(left), nodeToExpr(right)) + case Token("<", left :: right:: Nil) => LessThan(nodeToExpr(left), nodeToExpr(right)) + case Token("<=", left :: right:: Nil) => LessThanOrEqual(nodeToExpr(left), nodeToExpr(right)) + case Token(LIKE(), left :: right:: Nil) => Like(nodeToExpr(left), nodeToExpr(right)) + case Token(RLIKE(), left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) + case Token(REGEXP(), left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) + case Token("TOK_FUNCTION", Token("TOK_ISNOTNULL", Nil) :: child :: Nil) => + IsNotNull(nodeToExpr(child)) + case Token("TOK_FUNCTION", Token("TOK_ISNULL", Nil) :: child :: Nil) => + IsNull(nodeToExpr(child)) + case Token("TOK_FUNCTION", Token(IN(), Nil) :: value :: list) => + In(nodeToExpr(value), list.map(nodeToExpr)) + case Token("TOK_FUNCTION", + Token(BETWEEN(), Nil) :: + kw :: + target :: + minValue :: + maxValue :: Nil) => + + val targetExpression = nodeToExpr(target) + 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)) + case Token(OR(), left :: right:: Nil) => Or(nodeToExpr(left), nodeToExpr(right)) + case Token(NOT(), child :: Nil) => Not(nodeToExpr(child)) + case Token("!", child :: Nil) => Not(nodeToExpr(child)) + + /* Case statements */ + case Token("TOK_FUNCTION", Token(WHEN(), Nil) :: branches) => + CaseWhen(branches.map(nodeToExpr)) + case Token("TOK_FUNCTION", Token(CASE(), Nil) :: branches) => + val keyExpr = nodeToExpr(branches.head) + CaseKeyWhen(keyExpr, branches.drop(1).map(nodeToExpr)) + + /* Complex datatype manipulation */ + case Token("[", child :: ordinal :: Nil) => + UnresolvedExtractValue(nodeToExpr(child), nodeToExpr(ordinal)) + + /* Window Functions */ + case Token(text, args :+ Token("TOK_WINDOWSPEC", spec)) => + val function = nodeToExpr(node.copy(children = node.children.init)) + nodesToWindowSpecification(spec) match { + case reference: WindowSpecReference => + UnresolvedWindowExpression(function, reference) + case definition: WindowSpecDefinition => + WindowExpression(function, definition) + } + + /* UDFs - Must be last otherwise will preempt built in functions */ + case Token("TOK_FUNCTION", Token(name, Nil) :: args) => + UnresolvedFunction(name, args.map(nodeToExpr), isDistinct = false) + // Aggregate function with DISTINCT keyword. + case Token("TOK_FUNCTIONDI", Token(name, Nil) :: args) => + UnresolvedFunction(name, args.map(nodeToExpr), isDistinct = true) + case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: args) => + UnresolvedFunction(name, UnresolvedStar(None) :: Nil, isDistinct = false) + + /* Literals */ + case Token("TOK_NULL", Nil) => Literal.create(null, NullType) + case Token(TRUE(), Nil) => Literal.create(true, BooleanType) + case Token(FALSE(), Nil) => Literal.create(false, BooleanType) + case Token("TOK_STRINGLITERALSEQUENCE", strings) => + Literal(strings.map(s => ParseUtils.unescapeSQLString(s.text)).mkString) + + // This code is adapted from + // /ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java#L223 + case ast: ASTNode if numericAstTypes contains ast.tokenType => + var v: Literal = null + try { + if (ast.text.endsWith("L")) { + // Literal bigint. + v = Literal.create(ast.text.substring(0, ast.text.length() - 1).toLong, LongType) + } else if (ast.text.endsWith("S")) { + // Literal smallint. + v = Literal.create(ast.text.substring(0, ast.text.length() - 1).toShort, ShortType) + } else if (ast.text.endsWith("Y")) { + // Literal tinyint. + v = Literal.create(ast.text.substring(0, ast.text.length() - 1).toByte, ByteType) + } else if (ast.text.endsWith("BD") || ast.text.endsWith("D")) { + // Literal decimal + val strVal = ast.text.stripSuffix("D").stripSuffix("B") + v = Literal(Decimal(strVal)) + } else { + v = Literal.create(ast.text.toDouble, DoubleType) + v = Literal.create(ast.text.toLong, LongType) + v = Literal.create(ast.text.toInt, IntegerType) + } + } catch { + case nfe: NumberFormatException => // Do nothing + } + + if (v == null) { + sys.error(s"Failed to parse number '${ast.text}'.") + } else { + v + } + + case ast: ASTNode if ast.tokenType == SparkSqlParser.StringLiteral => + Literal(ParseUtils.unescapeSQLString(ast.text)) + + case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_DATELITERAL => + Literal(Date.valueOf(ast.text.substring(1, ast.text.length - 1))) + + case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_CHARSETLITERAL => + Literal(ParseUtils.charSetString(ast.children.head.text, ast.children(1).text)) + + case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_YEAR_MONTH_LITERAL => + Literal(CalendarInterval.fromYearMonthString(ast.text)) + + case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_DAY_TIME_LITERAL => + Literal(CalendarInterval.fromDayTimeString(ast.text)) + + case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_YEAR_LITERAL => + Literal(CalendarInterval.fromSingleUnitString("year", ast.text)) + + case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_MONTH_LITERAL => + Literal(CalendarInterval.fromSingleUnitString("month", ast.text)) + + case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_DAY_LITERAL => + Literal(CalendarInterval.fromSingleUnitString("day", ast.text)) + + case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_HOUR_LITERAL => + Literal(CalendarInterval.fromSingleUnitString("hour", ast.text)) + + case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_MINUTE_LITERAL => + Literal(CalendarInterval.fromSingleUnitString("minute", ast.text)) + + case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_SECOND_LITERAL => + Literal(CalendarInterval.fromSingleUnitString("second", ast.text)) + + case _ => + noParseRule("Expression", node) + } + + /* Case insensitive matches for Window Specification */ + val PRECEDING = "(?i)preceding".r + val FOLLOWING = "(?i)following".r + val CURRENT = "(?i)current".r + protected def nodesToWindowSpecification(nodes: Seq[ASTNode]): WindowSpec = nodes match { + case Token(windowName, Nil) :: Nil => + // Refer to a window spec defined in the window clause. + WindowSpecReference(windowName) + case Nil => + // OVER() + WindowSpecDefinition( + partitionSpec = Nil, + orderSpec = Nil, + frameSpecification = UnspecifiedFrame) + case spec => + val (partitionClause :: rowFrame :: rangeFrame :: Nil) = + getClauses( + Seq( + "TOK_PARTITIONINGSPEC", + "TOK_WINDOWRANGE", + "TOK_WINDOWVALUES"), + spec) + + // Handle Partition By and Order By. + val (partitionSpec, orderSpec) = partitionClause.map { partitionAndOrdering => + val (partitionByClause :: orderByClause :: sortByClause :: clusterByClause :: Nil) = + getClauses( + Seq("TOK_DISTRIBUTEBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_CLUSTERBY"), + partitionAndOrdering.children) + + (partitionByClause, orderByClause.orElse(sortByClause), clusterByClause) match { + case (Some(partitionByExpr), Some(orderByExpr), None) => + (partitionByExpr.children.map(nodeToExpr), + orderByExpr.children.map(nodeToSortOrder)) + case (Some(partitionByExpr), None, None) => + (partitionByExpr.children.map(nodeToExpr), Nil) + case (None, Some(orderByExpr), None) => + (Nil, orderByExpr.children.map(nodeToSortOrder)) + case (None, None, Some(clusterByExpr)) => + val expressions = clusterByExpr.children.map(nodeToExpr) + (expressions, expressions.map(SortOrder(_, Ascending))) + case _ => + noParseRule("Partition & Ordering", partitionAndOrdering) + } + }.getOrElse { + (Nil, Nil) + } + + // Handle Window Frame + val windowFrame = + if (rowFrame.isEmpty && rangeFrame.isEmpty) { + UnspecifiedFrame + } else { + val frameType = rowFrame.map(_ => RowFrame).getOrElse(RangeFrame) + def nodeToBoundary(node: ASTNode): FrameBoundary = node match { + case Token(PRECEDING(), Token(count, Nil) :: Nil) => + if (count.toLowerCase() == "unbounded") { + UnboundedPreceding + } else { + ValuePreceding(count.toInt) + } + case Token(FOLLOWING(), Token(count, Nil) :: Nil) => + if (count.toLowerCase() == "unbounded") { + UnboundedFollowing + } else { + ValueFollowing(count.toInt) + } + case Token(CURRENT(), Nil) => CurrentRow + case _ => + noParseRule("Window Frame Boundary", node) + } + + rowFrame.orElse(rangeFrame).map { frame => + frame.children match { + case precedingNode :: followingNode :: Nil => + SpecifiedWindowFrame( + frameType, + nodeToBoundary(precedingNode), + nodeToBoundary(followingNode)) + case precedingNode :: Nil => + SpecifiedWindowFrame(frameType, nodeToBoundary(precedingNode), CurrentRow) + case _ => + noParseRule("Window Frame", frame) + } + }.getOrElse(sys.error(s"If you see this, please file a bug report with your query.")) + } + + WindowSpecDefinition(partitionSpec, orderSpec, windowFrame) + } + + protected def nodeToTransformation( + node: ASTNode, + child: LogicalPlan): Option[ScriptTransformation] = None + + val explode = "(?i)explode".r + val jsonTuple = "(?i)json_tuple".r + protected def nodeToGenerate(node: ASTNode, outer: Boolean, child: LogicalPlan): Generate = { + val Token("TOK_SELECT", Token("TOK_SELEXPR", clauses) :: Nil) = node + + val alias = getClause("TOK_TABALIAS", clauses).children.head.text + + val generator = clauses.head match { + case Token("TOK_FUNCTION", Token(explode(), Nil) :: childNode :: Nil) => + Explode(nodeToExpr(childNode)) + case Token("TOK_FUNCTION", Token(jsonTuple(), Nil) :: children) => + JsonTuple(children.map(nodeToExpr)) + case other => + nodeToGenerator(other) + } + + val attributes = clauses.collect { + case Token(a, Nil) => UnresolvedAttribute(a.toLowerCase) + } + + Generate(generator, join = true, outer = outer, Some(alias.toLowerCase), attributes, child) + } + + protected def nodeToGenerator(node: ASTNode): Generator = noParseRule("Generator", node) + + protected def noParseRule(msg: String, node: ASTNode): Nothing = throw new NotImplementedError( + s"[$msg]: No parse rules for ASTNode type: ${node.tokenType}, tree:\n${node.treeString}") +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ASTNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ASTNode.scala new file mode 100644 index 0000000000000..ec5e71042d4be --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ASTNode.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.catalyst.parser + +import org.antlr.runtime.{Token, TokenRewriteStream} + +import org.apache.spark.sql.catalyst.trees.{Origin, TreeNode} + +case class ASTNode( + token: Token, + startIndex: Int, + stopIndex: Int, + children: List[ASTNode], + stream: TokenRewriteStream) extends TreeNode[ASTNode] { + /** Cache the number of children. */ + val numChildren = children.size + + /** tuple used in pattern matching. */ + val pattern = Some((token.getText, children)) + + /** Line in which the ASTNode starts. */ + lazy val line: Int = { + val line = token.getLine + if (line == 0) { + if (children.nonEmpty) children.head.line + else 0 + } else { + line + } + } + + /** Position of the Character at which ASTNode starts. */ + lazy val positionInLine: Int = { + val line = token.getCharPositionInLine + if (line == -1) { + if (children.nonEmpty) children.head.positionInLine + else 0 + } else { + line + } + } + + /** Origin of the ASTNode. */ + override val origin = Origin(Some(line), Some(positionInLine)) + + /** Source text. */ + lazy val source = stream.toString(startIndex, stopIndex) + + def text: String = token.getText + + def tokenType: Int = token.getType + + /** + * Checks if this node is equal to another node. + * + * Right now this function only checks the name, type, text and children of the node + * for equality. + */ + def treeEquals(other: ASTNode): Boolean = { + def check(f: ASTNode => Any): Boolean = { + val l = f(this) + val r = f(other) + (l == null && r == null) || l.equals(r) + } + if (other == null) { + false + } else if (!check(_.token.getType) + || !check(_.token.getText) + || !check(_.numChildren)) { + false + } else { + children.zip(other.children).forall { + case (l, r) => l treeEquals r + } + } + } + + override def simpleString: String = s"$text $line, $startIndex, $stopIndex, $positionInLine " +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala new file mode 100644 index 0000000000000..0e93af8b92cd2 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.parser + +import org.antlr.runtime._ +import org.antlr.runtime.tree.CommonTree + +import org.apache.spark.Logging +import org.apache.spark.sql.AnalysisException + +/** + * The ParseDriver takes a SQL command and turns this into an AST. + * + * This is based on Hive's org.apache.hadoop.hive.ql.parse.ParseDriver + */ +object ParseDriver extends Logging { + def parse(command: String, conf: ParserConf): ASTNode = { + logInfo(s"Parsing command: $command") + + // Setup error collection. + val reporter = new ParseErrorReporter() + + // Create lexer. + val lexer = new SparkSqlLexer(new ANTLRNoCaseStringStream(command)) + val tokens = new TokenRewriteStream(lexer) + lexer.configure(conf, reporter) + + // Create the parser. + val parser = new SparkSqlParser(tokens) + parser.configure(conf, reporter) + + try { + val result = parser.statement() + + // Check errors. + reporter.checkForErrors() + + // Return the AST node from the result. + logInfo(s"Parse completed.") + + // Find the non null token tree in the result. + def nonNullToken(tree: CommonTree): CommonTree = { + if (tree.token != null || tree.getChildCount == 0) tree + else nonNullToken(tree.getChild(0).asInstanceOf[CommonTree]) + } + val tree = nonNullToken(result.getTree) + + // Make sure all boundaries are set. + tree.setUnknownTokenBoundaries() + + // Construct the immutable AST. + def createASTNode(tree: CommonTree): ASTNode = { + val children = (0 until tree.getChildCount).map { i => + createASTNode(tree.getChild(i).asInstanceOf[CommonTree]) + }.toList + ASTNode(tree.token, tree.getTokenStartIndex, tree.getTokenStopIndex, children, tokens) + } + createASTNode(tree) + } + catch { + case e: RecognitionException => + logInfo(s"Parse failed.") + reporter.throwError(e) + } + } +} + +/** + * This string stream provides the lexer with upper case characters only. This greatly simplifies + * lexing the stream, while we can maintain the original command. + * + * This is based on Hive's org.apache.hadoop.hive.ql.parse.ParseDriver.ANTLRNoCaseStringStream + * + * The comment below (taken from the original class) describes the rationale for doing this: + * + * This class provides and implementation for a case insensitive token checker for the lexical + * analysis part of antlr. By converting the token stream into upper case at the time when lexical + * rules are checked, this class ensures that the lexical rules need to just match the token with + * upper case letters as opposed to combination of upper case and lower case characters. This is + * purely used for matching lexical rules. The actual token text is stored in the same way as the + * user input without actually converting it into an upper case. The token values are generated by + * the consume() function of the super class ANTLRStringStream. The LA() function is the lookahead + * function and is purely used for matching lexical rules. This also means that the grammar will + * only accept capitalized tokens in case it is run from other tools like antlrworks which do not + * have the ANTLRNoCaseStringStream implementation. + */ + +private[parser] class ANTLRNoCaseStringStream(input: String) extends ANTLRStringStream(input) { + override def LA(i: Int): Int = { + val la = super.LA(i) + if (la == 0 || la == CharStream.EOF) la + else Character.toUpperCase(la) + } +} + +/** + * Utility used by the Parser and the Lexer for error collection and reporting. + */ +private[parser] class ParseErrorReporter { + val errors = scala.collection.mutable.Buffer.empty[ParseError] + + def report(br: BaseRecognizer, re: RecognitionException, tokenNames: Array[String]): Unit = { + errors += ParseError(br, re, tokenNames) + } + + def checkForErrors(): Unit = { + if (errors.nonEmpty) { + val first = errors.head + val e = first.re + throwError(e.line, e.charPositionInLine, first.buildMessage().toString, errors.tail) + } + } + + def throwError(e: RecognitionException): Nothing = { + throwError(e.line, e.charPositionInLine, e.toString, errors) + } + + private def throwError( + line: Int, + startPosition: Int, + msg: String, + errors: Seq[ParseError]): Nothing = { + val b = new StringBuilder + b.append(msg).append("\n") + errors.foreach(error => error.buildMessage(b).append("\n")) + throw new AnalysisException(b.toString, Option(line), Option(startPosition)) + } +} + +/** + * Error collected during the parsing process. + * + * This is based on Hive's org.apache.hadoop.hive.ql.parse.ParseError + */ +private[parser] case class ParseError( + br: BaseRecognizer, + re: RecognitionException, + tokenNames: Array[String]) { + def buildMessage(s: StringBuilder = new StringBuilder): StringBuilder = { + s.append(br.getErrorHeader(re)).append(" ").append(br.getErrorMessage(re, tokenNames)) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserConf.scala new file mode 100644 index 0000000000000..ce449b11431a5 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserConf.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.catalyst.parser + +trait ParserConf { + def supportQuotedId: Boolean + def supportSQL11ReservedKeywords: Boolean +} + +case class SimpleParserConf( + supportQuotedId: Boolean = true, + supportSQL11ReservedKeywords: Boolean = false) extends ParserConf 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 b58a3739912bc..26c00dc250b4b 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 scala.collection.JavaConverters._ import org.apache.parquet.hadoop.ParquetOutputCommitter import org.apache.spark.sql.catalyst.CatalystConf +import org.apache.spark.sql.catalyst.parser.ParserConf //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines the configuration options for Spark SQL. @@ -451,6 +452,19 @@ private[spark] object SQLConf { doc = "When true, we could use `datasource`.`path` as table in SQL query" ) + val PARSER_SUPPORT_QUOTEDID = booleanConf("spark.sql.parser.supportQuotedIdentifiers", + defaultValue = Some(true), + isPublic = false, + doc = "Whether to use quoted identifier.\n false: default(past) behavior. Implies only" + + "alphaNumeric and underscore are valid characters in identifiers.\n" + + " true: implies column names can contain any character.") + + val PARSER_SUPPORT_SQL11_RESERVED_KEYWORDS = booleanConf( + "spark.sql.parser.supportSQL11ReservedKeywords", + defaultValue = Some(false), + isPublic = false, + doc = "This flag should be set to true to enable support for SQL2011 reserved keywords.") + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" val EXTERNAL_SORT = "spark.sql.planner.externalSort" @@ -471,7 +485,7 @@ private[spark] object SQLConf { * * SQLConf is thread-safe (internally synchronized, so safe to be used in multiple threads). */ -private[sql] class SQLConf extends Serializable with CatalystConf { +private[sql] class SQLConf extends Serializable with CatalystConf with ParserConf { import SQLConf._ /** Only low degree of contention is expected for conf, thus NOT using ConcurrentHashMap. */ @@ -569,6 +583,10 @@ private[sql] class SQLConf extends Serializable with CatalystConf { private[spark] def runSQLOnFile: Boolean = getConf(RUN_SQL_ON_FILES) + def supportQuotedId: Boolean = getConf(PARSER_SUPPORT_QUOTEDID) + + def supportSQL11ReservedKeywords: Boolean = getConf(PARSER_SUPPORT_SQL11_RESERVED_KEYWORDS) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala new file mode 100644 index 0000000000000..a322688a259e2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala @@ -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.sql.execution + +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} +import org.apache.spark.sql.catalyst.{CatalystQl, TableIdentifier} + +private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends CatalystQl(conf) { + /** Check if a command should not be explained. */ + protected def isNoExplainCommand(command: String): Boolean = "TOK_DESCTABLE" == command + + protected override def nodeToPlan(node: ASTNode): LogicalPlan = { + node match { + // Just fake explain for any of the native commands. + case Token("TOK_EXPLAIN", explainArgs) if isNoExplainCommand(explainArgs.head.text) => + ExplainCommand(OneRowRelation) + + case Token("TOK_EXPLAIN", explainArgs) if "TOK_CREATETABLE" == explainArgs.head.text => + val Some(crtTbl) :: _ :: extended :: Nil = + getClauses(Seq("TOK_CREATETABLE", "FORMATTED", "EXTENDED"), explainArgs) + ExplainCommand(nodeToPlan(crtTbl), extended = extended.isDefined) + + case Token("TOK_EXPLAIN", explainArgs) => + // Ignore FORMATTED if present. + val Some(query) :: _ :: extended :: Nil = + getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs) + ExplainCommand(nodeToPlan(query), extended = extended.isDefined) + + case Token("TOK_DESCTABLE", describeArgs) => + // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL + val Some(tableType) :: formatted :: extended :: pretty :: Nil = + getClauses(Seq("TOK_TABTYPE", "FORMATTED", "EXTENDED", "PRETTY"), describeArgs) + if (formatted.isDefined || pretty.isDefined) { + // FORMATTED and PRETTY are not supported and this statement will be treated as + // a Hive native command. + nodeToDescribeFallback(node) + } else { + tableType match { + case Token("TOK_TABTYPE", Token("TOK_TABNAME", nameParts :: Nil) :: Nil) => + nameParts match { + case Token(".", dbName :: tableName :: Nil) => + // It is describing a table with the format like "describe db.table". + // TODO: Actually, a user may mean tableName.columnName. Need to resolve this + // issue. + val tableIdent = extractTableIdent(nameParts) + datasources.DescribeCommand( + UnresolvedRelation(tableIdent, None), isExtended = extended.isDefined) + case Token(".", dbName :: tableName :: colName :: Nil) => + // It is describing a column with the format like "describe db.table column". + nodeToDescribeFallback(node) + case tableName => + // It is describing a table with the format like "describe table". + datasources.DescribeCommand( + UnresolvedRelation(TableIdentifier(tableName.text), None), + isExtended = extended.isDefined) + } + // All other cases. + case _ => nodeToDescribeFallback(node) + } + } + + case _ => + super.nodeToPlan(node) + } + } + + protected def nodeToDescribeFallback(node: ASTNode): LogicalPlan = noParseRule("Describe", node) +} diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index ffabb92179a18..cd0c2aeb93a9f 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -262,26 +262,6 @@ - - - org.antlr - antlr3-maven-plugin - - - - antlr - - - - - ${basedir}/src/main/antlr3 - - **/SparkSqlLexer.g - **/SparkSqlParser.g - - - - diff --git a/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTErrorNode.java b/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTErrorNode.java deleted file mode 100644 index 35ecdc5ad10a9..0000000000000 --- a/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTErrorNode.java +++ /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.parser; - -import org.antlr.runtime.RecognitionException; -import org.antlr.runtime.Token; -import org.antlr.runtime.TokenStream; -import org.antlr.runtime.tree.CommonErrorNode; - -public class ASTErrorNode extends ASTNode { - - /** - * - */ - private static final long serialVersionUID = 1L; - CommonErrorNode delegate; - - public ASTErrorNode(TokenStream input, Token start, Token stop, - RecognitionException e){ - delegate = new CommonErrorNode(input,start,stop,e); - } - - @Override - public boolean isNil() { return delegate.isNil(); } - - @Override - public int getType() { return delegate.getType(); } - - @Override - public String getText() { return delegate.getText(); } - @Override - public String toString() { return delegate.toString(); } -} diff --git a/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTNode.java b/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTNode.java deleted file mode 100644 index 33d9322b628ec..0000000000000 --- a/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTNode.java +++ /dev/null @@ -1,245 +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.parser; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - -import org.antlr.runtime.Token; -import org.antlr.runtime.tree.CommonTree; -import org.antlr.runtime.tree.Tree; -import org.apache.hadoop.hive.ql.lib.Node; - -public class ASTNode extends CommonTree implements Node, Serializable { - private static final long serialVersionUID = 1L; - private transient StringBuffer astStr; - private transient int startIndx = -1; - private transient int endIndx = -1; - private transient ASTNode rootNode; - private transient boolean isValidASTStr; - - public ASTNode() { - } - - /** - * Constructor. - * - * @param t - * Token for the CommonTree Node - */ - public ASTNode(Token t) { - super(t); - } - - public ASTNode(ASTNode node) { - super(node); - } - - @Override - public Tree dupNode() { - return new ASTNode(this); - } - - /* - * (non-Javadoc) - * - * @see org.apache.hadoop.hive.ql.lib.Node#getChildren() - */ - @Override - public ArrayList getChildren() { - if (super.getChildCount() == 0) { - return null; - } - - ArrayList ret_vec = new ArrayList(); - for (int i = 0; i < super.getChildCount(); ++i) { - ret_vec.add((Node) super.getChild(i)); - } - - return ret_vec; - } - - /* - * (non-Javadoc) - * - * @see org.apache.hadoop.hive.ql.lib.Node#getName() - */ - @Override - public String getName() { - return (Integer.valueOf(super.getToken().getType())).toString(); - } - - public String dump() { - StringBuilder sb = new StringBuilder("\n"); - dump(sb, ""); - return sb.toString(); - } - - private StringBuilder dump(StringBuilder sb, String ws) { - sb.append(ws); - sb.append(toString()); - sb.append("\n"); - - ArrayList children = getChildren(); - if (children != null) { - for (Node node : getChildren()) { - if (node instanceof ASTNode) { - ((ASTNode) node).dump(sb, ws + " "); - } else { - sb.append(ws); - sb.append(" NON-ASTNODE!!"); - sb.append("\n"); - } - } - } - return sb; - } - - private ASTNode getRootNodeWithValidASTStr(boolean useMemoizedRoot) { - if (useMemoizedRoot && rootNode != null && rootNode.parent == null && - rootNode.hasValidMemoizedString()) { - return rootNode; - } - ASTNode retNode = this; - while (retNode.parent != null) { - retNode = (ASTNode) retNode.parent; - } - rootNode=retNode; - if (!rootNode.isValidASTStr) { - rootNode.astStr = new StringBuffer(); - rootNode.toStringTree(rootNode); - rootNode.isValidASTStr = true; - } - return retNode; - } - - private boolean hasValidMemoizedString() { - return isValidASTStr && astStr != null; - } - - private void resetRootInformation() { - // Reset the previously stored rootNode string - if (rootNode != null) { - rootNode.astStr = null; - rootNode.isValidASTStr = false; - } - } - - private int getMemoizedStringLen() { - return astStr == null ? 0 : astStr.length(); - } - - private String getMemoizedSubString(int start, int end) { - return (astStr == null || start < 0 || end > astStr.length() || start >= end) ? null : - astStr.subSequence(start, end).toString(); - } - - private void addtoMemoizedString(String string) { - if (astStr == null) { - astStr = new StringBuffer(); - } - astStr.append(string); - } - - @Override - public void setParent(Tree t) { - super.setParent(t); - resetRootInformation(); - } - - @Override - public void addChild(Tree t) { - super.addChild(t); - resetRootInformation(); - } - - @Override - public void addChildren(List kids) { - super.addChildren(kids); - resetRootInformation(); - } - - @Override - public void setChild(int i, Tree t) { - super.setChild(i, t); - resetRootInformation(); - } - - @Override - public void insertChild(int i, Object t) { - super.insertChild(i, t); - resetRootInformation(); - } - - @Override - public Object deleteChild(int i) { - Object ret = super.deleteChild(i); - resetRootInformation(); - return ret; - } - - @Override - public void replaceChildren(int startChildIndex, int stopChildIndex, Object t) { - super.replaceChildren(startChildIndex, stopChildIndex, t); - resetRootInformation(); - } - - @Override - public String toStringTree() { - - // The root might have changed because of tree modifications. - // Compute the new root for this tree and set the astStr. - getRootNodeWithValidASTStr(true); - - // If rootNotModified is false, then startIndx and endIndx will be stale. - if (startIndx >= 0 && endIndx <= rootNode.getMemoizedStringLen()) { - return rootNode.getMemoizedSubString(startIndx, endIndx); - } - return toStringTree(rootNode); - } - - private String toStringTree(ASTNode rootNode) { - this.rootNode = rootNode; - startIndx = rootNode.getMemoizedStringLen(); - // Leaf node - if ( children==null || children.size()==0 ) { - rootNode.addtoMemoizedString(this.toString()); - endIndx = rootNode.getMemoizedStringLen(); - return this.toString(); - } - if ( !isNil() ) { - rootNode.addtoMemoizedString("("); - rootNode.addtoMemoizedString(this.toString()); - rootNode.addtoMemoizedString(" "); - } - for (int i = 0; children!=null && i < children.size(); i++) { - ASTNode t = (ASTNode)children.get(i); - if ( i>0 ) { - rootNode.addtoMemoizedString(" "); - } - t.toStringTree(rootNode); - } - if ( !isNil() ) { - rootNode.addtoMemoizedString(")"); - } - endIndx = rootNode.getMemoizedStringLen(); - return rootNode.getMemoizedSubString(startIndx, endIndx); - } -} diff --git a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseDriver.java b/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseDriver.java deleted file mode 100644 index c77198b087cbd..0000000000000 --- a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseDriver.java +++ /dev/null @@ -1,213 +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.parser; - -import java.util.ArrayList; -import org.antlr.runtime.ANTLRStringStream; -import org.antlr.runtime.CharStream; -import org.antlr.runtime.NoViableAltException; -import org.antlr.runtime.RecognitionException; -import org.antlr.runtime.Token; -import org.antlr.runtime.TokenRewriteStream; -import org.antlr.runtime.TokenStream; -import org.antlr.runtime.tree.CommonTree; -import org.antlr.runtime.tree.CommonTreeAdaptor; -import org.antlr.runtime.tree.TreeAdaptor; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.ql.Context; - -/** - * ParseDriver. - * - */ -public class ParseDriver { - - private static final Logger LOG = LoggerFactory.getLogger("hive.ql.parse.ParseDriver"); - - /** - * ANTLRNoCaseStringStream. - * - */ - //This class provides and implementation for a case insensitive token checker - //for the lexical analysis part of antlr. By converting the token stream into - //upper case at the time when lexical rules are checked, this class ensures that the - //lexical rules need to just match the token with upper case letters as opposed to - //combination of upper case and lower case characters. This is purely used for matching lexical - //rules. The actual token text is stored in the same way as the user input without - //actually converting it into an upper case. The token values are generated by the consume() - //function of the super class ANTLRStringStream. The LA() function is the lookahead function - //and is purely used for matching lexical rules. This also means that the grammar will only - //accept capitalized tokens in case it is run from other tools like antlrworks which - //do not have the ANTLRNoCaseStringStream implementation. - public class ANTLRNoCaseStringStream extends ANTLRStringStream { - - public ANTLRNoCaseStringStream(String input) { - super(input); - } - - @Override - public int LA(int i) { - - int returnChar = super.LA(i); - if (returnChar == CharStream.EOF) { - return returnChar; - } else if (returnChar == 0) { - return returnChar; - } - - return Character.toUpperCase((char) returnChar); - } - } - - /** - * HiveLexerX. - * - */ - public class HiveLexerX extends SparkSqlLexer { - - private final ArrayList errors; - - public HiveLexerX(CharStream input) { - super(input); - errors = new ArrayList(); - } - - @Override - public void displayRecognitionError(String[] tokenNames, RecognitionException e) { - errors.add(new ParseError(this, e, tokenNames)); - } - - @Override - public String getErrorMessage(RecognitionException e, String[] tokenNames) { - String msg = null; - - if (e instanceof NoViableAltException) { - // @SuppressWarnings("unused") - // NoViableAltException nvae = (NoViableAltException) e; - // for development, can add - // "decision=<<"+nvae.grammarDecisionDescription+">>" - // and "(decision="+nvae.decisionNumber+") and - // "state "+nvae.stateNumber - msg = "character " + getCharErrorDisplay(e.c) + " not supported here"; - } else { - msg = super.getErrorMessage(e, tokenNames); - } - - return msg; - } - - public ArrayList getErrors() { - return errors; - } - - } - - /** - * Tree adaptor for making antlr return ASTNodes instead of CommonTree nodes - * so that the graph walking algorithms and the rules framework defined in - * ql.lib can be used with the AST Nodes. - */ - public static final TreeAdaptor adaptor = new CommonTreeAdaptor() { - /** - * Creates an ASTNode for the given token. The ASTNode is a wrapper around - * antlr's CommonTree class that implements the Node interface. - * - * @param payload - * The token. - * @return Object (which is actually an ASTNode) for the token. - */ - @Override - public Object create(Token payload) { - return new ASTNode(payload); - } - - @Override - public Object dupNode(Object t) { - - return create(((CommonTree)t).token); - }; - - @Override - public Object errorNode(TokenStream input, Token start, Token stop, RecognitionException e) { - return new ASTErrorNode(input, start, stop, e); - }; - }; - - public ASTNode parse(String command) throws ParseException { - return parse(command, null); - } - - public ASTNode parse(String command, Context ctx) - throws ParseException { - return parse(command, ctx, true); - } - - /** - * Parses a command, optionally assigning the parser's token stream to the - * given context. - * - * @param command - * command to parse - * - * @param ctx - * context with which to associate this parser's token stream, or - * null if either no context is available or the context already has - * an existing stream - * - * @return parsed AST - */ - public ASTNode parse(String command, Context ctx, boolean setTokenRewriteStream) - throws ParseException { - LOG.info("Parsing command: " + command); - - HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command)); - TokenRewriteStream tokens = new TokenRewriteStream(lexer); - if (ctx != null) { - if ( setTokenRewriteStream) { - ctx.setTokenRewriteStream(tokens); - } - lexer.setHiveConf(ctx.getConf()); - } - SparkSqlParser parser = new SparkSqlParser(tokens); - if (ctx != null) { - parser.setHiveConf(ctx.getConf()); - } - parser.setTreeAdaptor(adaptor); - SparkSqlParser.statement_return r = null; - try { - r = parser.statement(); - } catch (RecognitionException e) { - e.printStackTrace(); - throw new ParseException(parser.errors); - } - - if (lexer.getErrors().size() == 0 && parser.errors.size() == 0) { - LOG.info("Parse Completed"); - } else if (lexer.getErrors().size() != 0) { - throw new ParseException(lexer.getErrors()); - } else { - throw new ParseException(parser.errors); - } - - ASTNode tree = (ASTNode) r.getTree(); - tree.setUnknownTokenBoundaries(); - return tree; - } -} diff --git a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseError.java b/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseError.java deleted file mode 100644 index b47bcfb2914df..0000000000000 --- a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseError.java +++ /dev/null @@ -1,54 +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.parser; - -import org.antlr.runtime.BaseRecognizer; -import org.antlr.runtime.RecognitionException; - -/** - * - */ -public class ParseError { - private final BaseRecognizer br; - private final RecognitionException re; - private final String[] tokenNames; - - ParseError(BaseRecognizer br, RecognitionException re, String[] tokenNames) { - this.br = br; - this.re = re; - this.tokenNames = tokenNames; - } - - BaseRecognizer getBaseRecognizer() { - return br; - } - - RecognitionException getRecognitionException() { - return re; - } - - String[] getTokenNames() { - return tokenNames; - } - - String getMessage() { - return br.getErrorHeader(re) + " " + br.getErrorMessage(re, tokenNames); - } - -} diff --git a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseException.java b/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseException.java deleted file mode 100644 index fff891ced5550..0000000000000 --- a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseException.java +++ /dev/null @@ -1,51 +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.parser; - -import java.util.ArrayList; - -/** - * ParseException. - * - */ -public class ParseException extends Exception { - - private static final long serialVersionUID = 1L; - ArrayList errors; - - public ParseException(ArrayList errors) { - super(); - this.errors = errors; - } - - @Override - public String getMessage() { - - StringBuilder sb = new StringBuilder(); - for (ParseError err : errors) { - if (sb.length() > 0) { - sb.append('\n'); - } - sb.append(err.getMessage()); - } - - return sb.toString(); - } - -} diff --git a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseUtils.java b/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseUtils.java deleted file mode 100644 index a5c2998f86cc1..0000000000000 --- a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseUtils.java +++ /dev/null @@ -1,96 +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.parser; - -import org.apache.hadoop.hive.common.type.HiveDecimal; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; - - -/** - * Library of utility functions used in the parse code. - * - */ -public final class ParseUtils { - /** - * Performs a descent of the leftmost branch of a tree, stopping when either a - * node with a non-null token is found or the leaf level is encountered. - * - * @param tree - * candidate node from which to start searching - * - * @return node at which descent stopped - */ - public static ASTNode findRootNonNullToken(ASTNode tree) { - while ((tree.getToken() == null) && (tree.getChildCount() > 0)) { - tree = (org.apache.spark.sql.parser.ASTNode) tree.getChild(0); - } - return tree; - } - - private ParseUtils() { - // prevent instantiation - } - - public static VarcharTypeInfo getVarcharTypeInfo(ASTNode node) - throws SemanticException { - if (node.getChildCount() != 1) { - throw new SemanticException("Bad params for type varchar"); - } - - String lengthStr = node.getChild(0).getText(); - return TypeInfoFactory.getVarcharTypeInfo(Integer.valueOf(lengthStr)); - } - - public static CharTypeInfo getCharTypeInfo(ASTNode node) - throws SemanticException { - if (node.getChildCount() != 1) { - throw new SemanticException("Bad params for type char"); - } - - String lengthStr = node.getChild(0).getText(); - return TypeInfoFactory.getCharTypeInfo(Integer.valueOf(lengthStr)); - } - - public static DecimalTypeInfo getDecimalTypeTypeInfo(ASTNode node) - throws SemanticException { - if (node.getChildCount() > 2) { - throw new SemanticException("Bad params for type decimal"); - } - - int precision = HiveDecimal.USER_DEFAULT_PRECISION; - int scale = HiveDecimal.USER_DEFAULT_SCALE; - - if (node.getChildCount() >= 1) { - String precStr = node.getChild(0).getText(); - precision = Integer.valueOf(precStr); - } - - if (node.getChildCount() == 2) { - String scaleStr = node.getChild(1).getText(); - scale = Integer.valueOf(scaleStr); - } - - return TypeInfoFactory.getDecimalTypeInfo(precision, scale); - } - -} diff --git a/sql/hive/src/main/java/org/apache/spark/sql/parser/SemanticAnalyzer.java b/sql/hive/src/main/java/org/apache/spark/sql/parser/SemanticAnalyzer.java deleted file mode 100644 index 4b2015e0df84e..0000000000000 --- a/sql/hive/src/main/java/org/apache/spark/sql/parser/SemanticAnalyzer.java +++ /dev/null @@ -1,406 +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.parser; - -import java.io.UnsupportedEncodingException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.antlr.runtime.tree.Tree; -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.serde.serdeConstants; -import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; - -/** - * SemanticAnalyzer. - * - */ -public abstract class SemanticAnalyzer { - public static String charSetString(String charSetName, String charSetString) - throws SemanticException { - try { - // The character set name starts with a _, so strip that - charSetName = charSetName.substring(1); - if (charSetString.charAt(0) == '\'') { - return new String(unescapeSQLString(charSetString).getBytes(), - charSetName); - } else // hex input is also supported - { - assert charSetString.charAt(0) == '0'; - assert charSetString.charAt(1) == 'x'; - charSetString = charSetString.substring(2); - - byte[] bArray = new byte[charSetString.length() / 2]; - int j = 0; - for (int i = 0; i < charSetString.length(); i += 2) { - int val = Character.digit(charSetString.charAt(i), 16) * 16 - + Character.digit(charSetString.charAt(i + 1), 16); - if (val > 127) { - val = val - 256; - } - bArray[j++] = (byte)val; - } - - String res = new String(bArray, charSetName); - return res; - } - } catch (UnsupportedEncodingException e) { - throw new SemanticException(e); - } - } - - /** - * Remove the encapsulating "`" pair from the identifier. We allow users to - * use "`" to escape identifier for table names, column names and aliases, in - * case that coincide with Hive language keywords. - */ - public static String unescapeIdentifier(String val) { - if (val == null) { - return null; - } - if (val.charAt(0) == '`' && val.charAt(val.length() - 1) == '`') { - val = val.substring(1, val.length() - 1); - } - return val; - } - - /** - * Converts parsed key/value properties pairs into a map. - * - * @param prop ASTNode parent of the key/value pairs - * - * @param mapProp property map which receives the mappings - */ - public static void readProps( - ASTNode prop, Map mapProp) { - - for (int propChild = 0; propChild < prop.getChildCount(); propChild++) { - String key = unescapeSQLString(prop.getChild(propChild).getChild(0) - .getText()); - String value = null; - if (prop.getChild(propChild).getChild(1) != null) { - value = unescapeSQLString(prop.getChild(propChild).getChild(1).getText()); - } - mapProp.put(key, value); - } - } - - private static final int[] multiplier = new int[] {1000, 100, 10, 1}; - - @SuppressWarnings("nls") - public static String unescapeSQLString(String b) { - Character enclosure = null; - - // Some of the strings can be passed in as unicode. For example, the - // delimiter can be passed in as \002 - So, we first check if the - // string is a unicode number, else go back to the old behavior - StringBuilder sb = new StringBuilder(b.length()); - for (int i = 0; i < b.length(); i++) { - - char currentChar = b.charAt(i); - if (enclosure == null) { - if (currentChar == '\'' || b.charAt(i) == '\"') { - enclosure = currentChar; - } - // ignore all other chars outside the enclosure - continue; - } - - if (enclosure.equals(currentChar)) { - enclosure = null; - continue; - } - - if (currentChar == '\\' && (i + 6 < b.length()) && b.charAt(i + 1) == 'u') { - int code = 0; - int base = i + 2; - for (int j = 0; j < 4; j++) { - int digit = Character.digit(b.charAt(j + base), 16); - code += digit * multiplier[j]; - } - sb.append((char)code); - i += 5; - continue; - } - - if (currentChar == '\\' && (i + 4 < b.length())) { - char i1 = b.charAt(i + 1); - char i2 = b.charAt(i + 2); - char i3 = b.charAt(i + 3); - if ((i1 >= '0' && i1 <= '1') && (i2 >= '0' && i2 <= '7') - && (i3 >= '0' && i3 <= '7')) { - byte bVal = (byte) ((i3 - '0') + ((i2 - '0') * 8) + ((i1 - '0') * 8 * 8)); - byte[] bValArr = new byte[1]; - bValArr[0] = bVal; - String tmp = new String(bValArr); - sb.append(tmp); - i += 3; - continue; - } - } - - if (currentChar == '\\' && (i + 2 < b.length())) { - char n = b.charAt(i + 1); - switch (n) { - case '0': - sb.append("\0"); - break; - case '\'': - sb.append("'"); - break; - case '"': - sb.append("\""); - break; - case 'b': - sb.append("\b"); - break; - case 'n': - sb.append("\n"); - break; - case 'r': - sb.append("\r"); - break; - case 't': - sb.append("\t"); - break; - case 'Z': - sb.append("\u001A"); - break; - case '\\': - sb.append("\\"); - break; - // The following 2 lines are exactly what MySQL does TODO: why do we do this? - case '%': - sb.append("\\%"); - break; - case '_': - sb.append("\\_"); - break; - default: - sb.append(n); - } - i++; - } else { - sb.append(currentChar); - } - } - return sb.toString(); - } - - /** - * Get the list of FieldSchema out of the ASTNode. - */ - public static List getColumns(ASTNode ast, boolean lowerCase) throws SemanticException { - List colList = new ArrayList(); - int numCh = ast.getChildCount(); - for (int i = 0; i < numCh; i++) { - FieldSchema col = new FieldSchema(); - ASTNode child = (ASTNode) ast.getChild(i); - Tree grandChild = child.getChild(0); - if(grandChild != null) { - String name = grandChild.getText(); - if(lowerCase) { - name = name.toLowerCase(); - } - // child 0 is the name of the column - col.setName(unescapeIdentifier(name)); - // child 1 is the type of the column - ASTNode typeChild = (ASTNode) (child.getChild(1)); - col.setType(getTypeStringFromAST(typeChild)); - - // child 2 is the optional comment of the column - if (child.getChildCount() == 3) { - col.setComment(unescapeSQLString(child.getChild(2).getText())); - } - } - colList.add(col); - } - return colList; - } - - protected static String getTypeStringFromAST(ASTNode typeNode) - throws SemanticException { - switch (typeNode.getType()) { - case SparkSqlParser.TOK_LIST: - return serdeConstants.LIST_TYPE_NAME + "<" - + getTypeStringFromAST((ASTNode) typeNode.getChild(0)) + ">"; - case SparkSqlParser.TOK_MAP: - return serdeConstants.MAP_TYPE_NAME + "<" - + getTypeStringFromAST((ASTNode) typeNode.getChild(0)) + "," - + getTypeStringFromAST((ASTNode) typeNode.getChild(1)) + ">"; - case SparkSqlParser.TOK_STRUCT: - return getStructTypeStringFromAST(typeNode); - case SparkSqlParser.TOK_UNIONTYPE: - return getUnionTypeStringFromAST(typeNode); - default: - return getTypeName(typeNode); - } - } - - private static String getStructTypeStringFromAST(ASTNode typeNode) - throws SemanticException { - String typeStr = serdeConstants.STRUCT_TYPE_NAME + "<"; - typeNode = (ASTNode) typeNode.getChild(0); - int children = typeNode.getChildCount(); - if (children <= 0) { - throw new SemanticException("empty struct not allowed."); - } - StringBuilder buffer = new StringBuilder(typeStr); - for (int i = 0; i < children; i++) { - ASTNode child = (ASTNode) typeNode.getChild(i); - buffer.append(unescapeIdentifier(child.getChild(0).getText())).append(":"); - buffer.append(getTypeStringFromAST((ASTNode) child.getChild(1))); - if (i < children - 1) { - buffer.append(","); - } - } - - buffer.append(">"); - return buffer.toString(); - } - - private static String getUnionTypeStringFromAST(ASTNode typeNode) - throws SemanticException { - String typeStr = serdeConstants.UNION_TYPE_NAME + "<"; - typeNode = (ASTNode) typeNode.getChild(0); - int children = typeNode.getChildCount(); - if (children <= 0) { - throw new SemanticException("empty union not allowed."); - } - StringBuilder buffer = new StringBuilder(typeStr); - for (int i = 0; i < children; i++) { - buffer.append(getTypeStringFromAST((ASTNode) typeNode.getChild(i))); - if (i < children - 1) { - buffer.append(","); - } - } - buffer.append(">"); - typeStr = buffer.toString(); - return typeStr; - } - - public static String getAstNodeText(ASTNode tree) { - return tree.getChildCount() == 0?tree.getText() : - getAstNodeText((ASTNode)tree.getChild(tree.getChildCount() - 1)); - } - - public static String generateErrorMessage(ASTNode ast, String message) { - StringBuilder sb = new StringBuilder(); - if (ast == null) { - sb.append(message).append(". Cannot tell the position of null AST."); - return sb.toString(); - } - sb.append(ast.getLine()); - sb.append(":"); - sb.append(ast.getCharPositionInLine()); - sb.append(" "); - sb.append(message); - sb.append(". Error encountered near token '"); - sb.append(getAstNodeText(ast)); - sb.append("'"); - return sb.toString(); - } - - private static final Map TokenToTypeName = new HashMap(); - - static { - TokenToTypeName.put(SparkSqlParser.TOK_BOOLEAN, serdeConstants.BOOLEAN_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_TINYINT, serdeConstants.TINYINT_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_SMALLINT, serdeConstants.SMALLINT_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_INT, serdeConstants.INT_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_BIGINT, serdeConstants.BIGINT_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_FLOAT, serdeConstants.FLOAT_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_DOUBLE, serdeConstants.DOUBLE_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_STRING, serdeConstants.STRING_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_CHAR, serdeConstants.CHAR_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_VARCHAR, serdeConstants.VARCHAR_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_BINARY, serdeConstants.BINARY_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_DATE, serdeConstants.DATE_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_DATETIME, serdeConstants.DATETIME_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_TIMESTAMP, serdeConstants.TIMESTAMP_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_INTERVAL_YEAR_MONTH, serdeConstants.INTERVAL_YEAR_MONTH_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_INTERVAL_DAY_TIME, serdeConstants.INTERVAL_DAY_TIME_TYPE_NAME); - TokenToTypeName.put(SparkSqlParser.TOK_DECIMAL, serdeConstants.DECIMAL_TYPE_NAME); - } - - public static String getTypeName(ASTNode node) throws SemanticException { - int token = node.getType(); - String typeName; - - // datetime type isn't currently supported - if (token == SparkSqlParser.TOK_DATETIME) { - throw new SemanticException(ErrorMsg.UNSUPPORTED_TYPE.getMsg()); - } - - switch (token) { - case SparkSqlParser.TOK_CHAR: - CharTypeInfo charTypeInfo = ParseUtils.getCharTypeInfo(node); - typeName = charTypeInfo.getQualifiedName(); - break; - case SparkSqlParser.TOK_VARCHAR: - VarcharTypeInfo varcharTypeInfo = ParseUtils.getVarcharTypeInfo(node); - typeName = varcharTypeInfo.getQualifiedName(); - break; - case SparkSqlParser.TOK_DECIMAL: - DecimalTypeInfo decTypeInfo = ParseUtils.getDecimalTypeTypeInfo(node); - typeName = decTypeInfo.getQualifiedName(); - break; - default: - typeName = TokenToTypeName.get(token); - } - return typeName; - } - - public static String relativeToAbsolutePath(HiveConf conf, String location) throws SemanticException { - boolean testMode = conf.getBoolVar(HiveConf.ConfVars.HIVETESTMODE); - if (testMode) { - URI uri = new Path(location).toUri(); - String scheme = uri.getScheme(); - String authority = uri.getAuthority(); - String path = uri.getPath(); - if (!path.startsWith("/")) { - path = (new Path(System.getProperty("test.tmp.dir"), - path)).toUri().getPath(); - } - if (StringUtils.isEmpty(scheme)) { - scheme = "pfile"; - } - try { - uri = new URI(scheme, authority, path, null, null); - } catch (URISyntaxException e) { - throw new SemanticException(ErrorMsg.INVALID_PATH.getMsg(), e); - } - return uri.toString(); - } else { - //no-op for non-test mode for now - return location; - } - } -} 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 31d82eb20f6e4..bf3fe12d5c5d2 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,41 +17,30 @@ package org.apache.spark.sql.hive -import java.sql.Date import java.util.Locale import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer +import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.ql.{Context, ErrorMsg} -import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} -import org.apache.hadoop.hive.ql.lib.Node -import org.apache.hadoop.hive.ql.parse.SemanticException -import org.apache.hadoop.hive.ql.plan.PlanUtils +import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, FunctionInfo} +import org.apache.hadoop.hive.ql.parse.EximUtil import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe - import org.apache.spark.Logging -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.parser._ +import org.apache.spark.sql.catalyst.parser.ParseUtils._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.trees.CurrentOrigin -import org.apache.spark.sql.execution.ExplainCommand -import org.apache.spark.sql.execution.datasources.DescribeCommand -import org.apache.spark.sql.hive.HiveShim._ +import org.apache.spark.sql.execution.SparkQl +import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper import org.apache.spark.sql.hive.client._ -import org.apache.spark.sql.hive.execution.{AnalyzeTable, DropTable, HiveNativeCommand, HiveScriptIOSchema} -import org.apache.spark.sql.parser._ +import org.apache.spark.sql.hive.execution.{HiveNativeCommand, AnalyzeTable, DropTable, HiveScriptIOSchema} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval -import org.apache.spark.util.random.RandomSampler +import org.apache.spark.sql.AnalysisException /** * Used when we need to start parsing the AST before deciding that we are going to pass the command @@ -71,7 +60,7 @@ private[hive] case class CreateTableAsSelect( override def output: Seq[Attribute] = Seq.empty[Attribute] override lazy val resolved: Boolean = tableDesc.specifiedDatabase.isDefined && - tableDesc.schema.size > 0 && + tableDesc.schema.nonEmpty && tableDesc.serde.isDefined && tableDesc.inputFormat.isDefined && tableDesc.outputFormat.isDefined && @@ -89,7 +78,7 @@ private[hive] case class CreateViewAsSelect( } /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ -private[hive] object HiveQl extends Logging { +private[hive] object HiveQl extends SparkQl with Logging { protected val nativeCommands = Seq( "TOK_ALTERDATABASE_OWNER", "TOK_ALTERDATABASE_PROPERTIES", @@ -180,103 +169,6 @@ private[hive] object HiveQl extends Logging { protected val hqlParser = new ExtendedHiveQlParser - /** - * A set of implicit transformations that allow Hive ASTNodes to be rewritten by transformations - * similar to [[catalyst.trees.TreeNode]]. - * - * Note that this should be considered very experimental and is not indented as a replacement - * for TreeNode. Primarily it should be noted ASTNodes are not immutable and do not appear to - * have clean copy semantics. Therefore, users of this class should take care when - * copying/modifying trees that might be used elsewhere. - */ - implicit class TransformableNode(n: ASTNode) { - /** - * Returns a copy of this node where `rule` has been recursively applied to it and all of its - * children. When `rule` does not apply to a given node it is left unchanged. - * @param rule the function use to transform this nodes children - */ - def transform(rule: PartialFunction[ASTNode, ASTNode]): ASTNode = { - try { - val afterRule = rule.applyOrElse(n, identity[ASTNode]) - afterRule.withChildren( - nilIfEmpty(afterRule.getChildren) - .asInstanceOf[Seq[ASTNode]] - .map(ast => Option(ast).map(_.transform(rule)).orNull)) - } catch { - case e: Exception => - logError(dumpTree(n).toString) - throw e - } - } - - /** - * Returns a scala.Seq equivalent to [s] or Nil if [s] is null. - */ - private def nilIfEmpty[A](s: java.util.List[A]): Seq[A] = - Option(s).map(_.asScala).getOrElse(Nil) - - /** - * Returns this ASTNode with the text changed to `newText`. - */ - def withText(newText: String): ASTNode = { - n.token.asInstanceOf[org.antlr.runtime.CommonToken].setText(newText) - n - } - - /** - * Returns this ASTNode with the children changed to `newChildren`. - */ - def withChildren(newChildren: Seq[ASTNode]): ASTNode = { - (1 to n.getChildCount).foreach(_ => n.deleteChild(0)) - newChildren.foreach(n.addChild(_)) - n - } - - /** - * Throws an error if this is not equal to other. - * - * Right now this function only checks the name, type, text and children of the node - * for equality. - */ - def checkEquals(other: ASTNode): Unit = { - def check(field: String, f: ASTNode => Any): Unit = if (f(n) != f(other)) { - sys.error(s"$field does not match for trees. " + - s"'${f(n)}' != '${f(other)}' left: ${dumpTree(n)}, right: ${dumpTree(other)}") - } - check("name", _.getName) - check("type", _.getType) - check("text", _.getText) - check("numChildren", n => nilIfEmpty(n.getChildren).size) - - val leftChildren = nilIfEmpty(n.getChildren).asInstanceOf[Seq[ASTNode]] - val rightChildren = nilIfEmpty(other.getChildren).asInstanceOf[Seq[ASTNode]] - leftChildren zip rightChildren foreach { - case (l, r) => l checkEquals r - } - } - } - - /** - * Returns the AST for the given SQL string. - */ - 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 = createContext() - val node = getAst(sql, hContext) - hContext.clear() - node - } - - private def createContext(): Context = new Context(hiveConf) - - private def getAst(sql: String, context: Context) = - ParseUtils.findRootNonNullToken( - (new ParseDriver).parse(sql, context)) - /** * Returns the HiveConf */ @@ -296,226 +188,16 @@ private[hive] object HiveQl extends Logging { /** Returns a LogicalPlan for a given HiveQL string. */ def parseSql(sql: String): LogicalPlan = hqlParser.parse(sql) - val errorRegEx = "line (\\d+):(\\d+) (.*)".r - - /** Creates LogicalPlan for a given HiveQL string. */ - def createPlan(sql: String): LogicalPlan = { - try { - val context = createContext() - val tree = getAst(sql, context) - val plan = if (nativeCommands contains tree.getText) { - HiveNativeCommand(sql) - } else { - nodeToPlan(tree, context) match { - case NativePlaceholder => HiveNativeCommand(sql) - case other => other - } - } - context.clear() - plan - } catch { - case pe: ParseException => - pe.getMessage match { - case errorRegEx(line, start, message) => - throw new AnalysisException(message, Some(line.toInt), Some(start.toInt)) - case otherMessage => - throw new AnalysisException(otherMessage) - } - case e: MatchError => throw e - case e: Exception => - throw new AnalysisException(e.getMessage) - case e: NotImplementedError => - throw new AnalysisException( - s""" - |Unsupported language features in query: $sql - |${dumpTree(getAst(sql))} - |$e - |${e.getStackTrace.head} - """.stripMargin) - } - } - - def parseDdl(ddl: String): Seq[Attribute] = { - val tree = - try { - ParseUtils.findRootNonNullToken( - (new ParseDriver) - .parse(ddl, null /* no context required for parsing alone */)) - } catch { - case pe: org.apache.hadoop.hive.ql.parse.ParseException => - throw new RuntimeException(s"Failed to parse ddl: '$ddl'", pe) - } - assert(tree.asInstanceOf[ASTNode].getText == "TOK_CREATETABLE", "Only CREATE TABLE supported.") - val tableOps = tree.getChildren - val colList = - tableOps.asScala - .find(_.asInstanceOf[ASTNode].getText == "TOK_TABCOLLIST") - .getOrElse(sys.error("No columnList!")).getChildren - - colList.asScala.map(nodeToAttribute) - } - - /** Extractor for matching Hive's AST Tokens. */ - private[hive] case class Token(name: String, children: Seq[ASTNode]) extends Node { - def getName(): String = name - def getChildren(): java.util.List[Node] = { - val col = new java.util.ArrayList[Node](children.size) - children.foreach(col.add(_)) - col - } - } - object Token { - /** @return matches of the form (tokenName, children). */ - def unapply(t: Any): Option[(String, Seq[ASTNode])] = t match { - case t: ASTNode => - CurrentOrigin.setPosition(t.getLine, t.getCharPositionInLine) - Some((t.getText, - Option(t.getChildren).map(_.asScala.toList).getOrElse(Nil).asInstanceOf[Seq[ASTNode]])) - case t: Token => Some((t.name, t.children)) - case _ => None - } - } - - protected def getClauses( - clauseNames: Seq[String], - nodeList: Seq[ASTNode]): Seq[Option[ASTNode]] = { - var remainingNodes = nodeList - val clauses = clauseNames.map { clauseName => - val (matches, nonMatches) = remainingNodes.partition(_.getText.toUpperCase == clauseName) - remainingNodes = nonMatches ++ (if (matches.nonEmpty) matches.tail else Nil) - matches.headOption - } - - 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 - } - - def getClause(clauseName: String, nodeList: Seq[Node]): Node = - getClauseOption(clauseName, nodeList).getOrElse(sys.error( - s"Expected clause $clauseName missing from ${nodeList.map(dumpTree(_)).mkString("\n")}")) - - def getClauseOption(clauseName: String, nodeList: Seq[Node]): Option[Node] = { - nodeList.filter { case ast: ASTNode => ast.getText == clauseName } match { - case Seq(oneMatch) => Some(oneMatch) - case Seq() => None - case _ => sys.error(s"Found multiple instances of clause $clauseName") - } - } - - protected def nodeToAttribute(node: Node): Attribute = node match { - case Token("TOK_TABCOL", Token(colName, Nil) :: dataType :: Nil) => - AttributeReference(colName, nodeToDataType(dataType), true)() - - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") - } - - protected def nodeToDataType(node: Node): DataType = node match { - 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.USER_DEFAULT - case Token("TOK_BIGINT", Nil) => LongType - case Token("TOK_INT", Nil) => IntegerType - case Token("TOK_TINYINT", Nil) => ByteType - 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 - case Token("TOK_TIMESTAMP", Nil) => TimestampType - case Token("TOK_BINARY", Nil) => BinaryType - case Token("TOK_LIST", elementType :: Nil) => ArrayType(nodeToDataType(elementType)) - case Token("TOK_STRUCT", - Token("TOK_TABCOLLIST", fields) :: Nil) => - StructType(fields.map(nodeToStructField)) - case Token("TOK_MAP", - keyType :: - valueType :: Nil) => - MapType(nodeToDataType(keyType), nodeToDataType(valueType)) - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for DataType:\n ${dumpTree(a).toString} ") - } - - protected def nodeToStructField(node: Node): StructField = node match { - case Token("TOK_TABCOL", - Token(fieldName, Nil) :: - dataType :: Nil) => - StructField(fieldName, nodeToDataType(dataType), nullable = true) - case Token("TOK_TABCOL", - Token(fieldName, Nil) :: - dataType :: - _ /* comment */:: Nil) => - StructField(fieldName, nodeToDataType(dataType), nullable = true) - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for StructField:\n ${dumpTree(a).toString} ") - } - - protected def extractTableIdent(tableNameParts: Node): TableIdentifier = { - tableNameParts.getChildren.asScala.map { - case Token(part, Nil) => cleanIdentifier(part) - } match { - case Seq(tableOnly) => TableIdentifier(tableOnly) - case Seq(databaseName, table) => TableIdentifier(table, Some(databaseName)) - case other => sys.error("Hive only supports tables names like 'tableName' " + - s"or 'databaseName.tableName', found '$other'") - } - } - - /** - * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) - * is equivalent to - * SELECT MAX(value) FROM src GROUP BY k1, k2 UNION SELECT MAX(value) FROM src GROUP BY k2 - * Check the following link for details. - * -https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C+Grouping+and+Rollup - * - * The bitmask denotes the grouping expressions validity for a grouping set, - * the bitmask also be called as grouping id (`GROUPING__ID`, the virtual column in Hive) - * e.g. In superset (k1, k2, k3), (bit 0: k1, bit 1: k2, and bit 2: k3), the grouping id of - * GROUPING SETS (k1, k2) and (k2) should be 3 and 2 respectively. - */ - protected def extractGroupingSet(children: Seq[ASTNode]): (Seq[Expression], Seq[Int]) = { - val (keyASTs, setASTs) = children.partition( n => n match { - case Token("TOK_GROUPING_SETS_EXPRESSION", children) => false // grouping sets - case _ => true // grouping keys - }) - - val keys = keyASTs.map(nodeToExpr).toSeq - val keyMap = keyASTs.map(_.toStringTree).zipWithIndex.toMap - - val bitmasks: Seq[Int] = setASTs.map(set => set match { - case Token("TOK_GROUPING_SETS_EXPRESSION", null) => 0 - case Token("TOK_GROUPING_SETS_EXPRESSION", children) => - children.foldLeft(0)((bitmap, col) => { - val colString = col.asInstanceOf[ASTNode].toStringTree() - require(keyMap.contains(colString), s"$colString doens't show up in the GROUP BY list") - bitmap | 1 << keyMap(colString) - }) - case _ => sys.error("Expect GROUPING SETS clause") - }) - - (keys, bitmasks) - } - - protected def getProperties(node: Node): Seq[(String, String)] = node match { + protected def getProperties(node: ASTNode): Seq[(String, String)] = node match { case Token("TOK_TABLEPROPLIST", list) => list.map { case Token("TOK_TABLEPROPERTY", Token(key, Nil) :: Token(value, Nil) :: Nil) => - (unquoteString(key) -> unquoteString(value)) + unquoteString(key) -> unquoteString(value) } } private def createView( view: ASTNode, - context: Context, viewNameParts: ASTNode, query: ASTNode, schema: Seq[HiveColumn], @@ -524,8 +206,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C replace: Boolean): CreateViewAsSelect = { val TableIdentifier(viewName, dbName) = extractTableIdent(viewNameParts) - val originalText = context.getTokenRewriteStream - .toString(query.getTokenStartIndex, query.getTokenStopIndex) + val originalText = query.source val tableDesc = HiveTable( specifiedDatabase = dbName, @@ -544,104 +225,67 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C // We need to keep the original SQL string so that if `spark.sql.nativeView` is // false, we can fall back to use hive native command later. // We can remove this when parser is configurable(can access SQLConf) in the future. - val sql = context.getTokenRewriteStream - .toString(view.getTokenStartIndex, view.getTokenStopIndex) - CreateViewAsSelect(tableDesc, nodeToPlan(query, context), allowExist, replace, sql) + val sql = view.source + CreateViewAsSelect(tableDesc, nodeToPlan(query), allowExist, replace, sql) } - protected def nodeToPlan(node: ASTNode, context: Context): 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) - // 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) + protected override def createPlan( + sql: String, + node: ASTNode): LogicalPlan = { + if (nativeCommands.contains(node.text)) { + HiveNativeCommand(sql) + } else { + nodeToPlan(node) match { + case NativePlaceholder => HiveNativeCommand(sql) + case plan => plan } - // Just fake explain for any of the native commands. - case Token("TOK_EXPLAIN", explainArgs) - if noExplainCommands.contains(explainArgs.head.getText) => - ExplainCommand(OneRowRelation) - case Token("TOK_EXPLAIN", explainArgs) - if "TOK_CREATETABLE" == explainArgs.head.getText => - val Some(crtTbl) :: _ :: extended :: Nil = - getClauses(Seq("TOK_CREATETABLE", "FORMATTED", "EXTENDED"), explainArgs) - ExplainCommand( - nodeToPlan(crtTbl, context), - extended = extended.isDefined) - case Token("TOK_EXPLAIN", explainArgs) => - // Ignore FORMATTED if present. - val Some(query) :: _ :: extended :: Nil = - getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs) - ExplainCommand( - nodeToPlan(query, context), - extended = extended.isDefined) - - case Token("TOK_DESCTABLE", describeArgs) => - // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL - val Some(tableType) :: formatted :: extended :: pretty :: Nil = - getClauses(Seq("TOK_TABTYPE", "FORMATTED", "EXTENDED", "PRETTY"), describeArgs) - if (formatted.isDefined || pretty.isDefined) { - // FORMATTED and PRETTY are not supported and this statement will be treated as - // a Hive native command. - NativePlaceholder - } else { - tableType match { - case Token("TOK_TABTYPE", Token("TOK_TABNAME", nameParts :: Nil) :: Nil) => { - nameParts match { - case Token(".", dbName :: tableName :: Nil) => - // It is describing a table with the format like "describe db.table". - // TODO: Actually, a user may mean tableName.columnName. Need to resolve this issue. - val tableIdent = extractTableIdent(nameParts) - DescribeCommand( - UnresolvedRelation(tableIdent, None), isExtended = extended.isDefined) - case Token(".", dbName :: tableName :: colName :: Nil) => - // It is describing a column with the format like "describe db.table column". - NativePlaceholder - case tableName => - // It is describing a table with the format like "describe table". - DescribeCommand( - UnresolvedRelation(TableIdentifier(tableName.getText), None), - isExtended = extended.isDefined) - } - } - // All other cases. - case _ => NativePlaceholder + } + } + + protected override def isNoExplainCommand(command: String): Boolean = + noExplainCommands.contains(command) + + protected override def nodeToPlan(node: ASTNode): 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) + + // 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) } - } - case view @ Token("TOK_ALTERVIEW", children) => - val Some(viewNameParts) :: maybeQuery :: ignores = - getClauses(Seq( - "TOK_TABNAME", - "TOK_QUERY", - "TOK_ALTERVIEW_ADDPARTS", - "TOK_ALTERVIEW_DROPPARTS", - "TOK_ALTERVIEW_PROPERTIES", - "TOK_ALTERVIEW_RENAME"), children) + case view @ Token("TOK_ALTERVIEW", children) => + val Some(nameParts) :: maybeQuery :: _ = + getClauses(Seq( + "TOK_TABNAME", + "TOK_QUERY", + "TOK_ALTERVIEW_ADDPARTS", + "TOK_ALTERVIEW_DROPPARTS", + "TOK_ALTERVIEW_PROPERTIES", + "TOK_ALTERVIEW_RENAME"), children) - // if ALTER VIEW doesn't have query part, let hive to handle it. - maybeQuery.map { query => - createView(view, context, viewNameParts, query, Nil, Map(), false, true) - }.getOrElse(NativePlaceholder) + // if ALTER VIEW doesn't have query part, let hive to handle it. + maybeQuery.map { query => + createView(view, nameParts, query, Nil, Map(), allowExist = false, replace = true) + }.getOrElse(NativePlaceholder) - case view @ Token("TOK_CREATEVIEW", children) + case view @ Token("TOK_CREATEVIEW", children) if children.collect { case t @ Token("TOK_QUERY", _) => t }.nonEmpty => - val Seq( + val Seq( Some(viewNameParts), Some(query), maybeComment, @@ -650,1224 +294,466 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C maybeProperties, maybeColumns, maybePartCols - ) = getClauses(Seq( - "TOK_TABNAME", - "TOK_QUERY", - "TOK_TABLECOMMENT", - "TOK_ORREPLACE", - "TOK_IFNOTEXISTS", - "TOK_TABLEPROPERTIES", - "TOK_TABCOLNAME", - "TOK_VIEWPARTCOLS"), children) - - // If the view is partitioned, we let hive handle it. - if (maybePartCols.isDefined) { - NativePlaceholder - } else { - val schema = maybeColumns.map { cols => - SemanticAnalyzer.getColumns(cols, true).asScala.map { field => + ) = getClauses(Seq( + "TOK_TABNAME", + "TOK_QUERY", + "TOK_TABLECOMMENT", + "TOK_ORREPLACE", + "TOK_IFNOTEXISTS", + "TOK_TABLEPROPERTIES", + "TOK_TABCOLNAME", + "TOK_VIEWPARTCOLS"), children) + + // If the view is partitioned, we let hive handle it. + if (maybePartCols.isDefined) { + NativePlaceholder + } else { + val schema = maybeColumns.map { cols => // We can't specify column types when create view, so fill it with null first, and // update it after the schema has been resolved later. - HiveColumn(field.getName, null, field.getComment) - } - }.getOrElse(Seq.empty[HiveColumn]) - - val properties = scala.collection.mutable.Map.empty[String, String] - - maybeProperties.foreach { - case Token("TOK_TABLEPROPERTIES", list :: Nil) => - properties ++= getProperties(list) - } - - maybeComment.foreach { - case Token("TOK_TABLECOMMENT", child :: Nil) => - val comment = SemanticAnalyzer.unescapeSQLString(child.getText) - if (comment ne null) { - properties += ("comment" -> comment) - } - } - - createView(view, context, viewNameParts, query, schema, properties.toMap, - allowExisting.isDefined, replace.isDefined) - } - - case Token("TOK_CREATETABLE", children) - if children.collect { case t @ Token("TOK_QUERY", _) => t }.nonEmpty => - // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL - val ( - Some(tableNameParts) :: - _ /* likeTable */ :: - externalTable :: - Some(query) :: - allowExisting +: - ignores) = - getClauses( - Seq( - "TOK_TABNAME", - "TOK_LIKETABLE", - "EXTERNAL", - "TOK_QUERY", - "TOK_IFNOTEXISTS", - "TOK_TABLECOMMENT", - "TOK_TABCOLLIST", - "TOK_TABLEPARTCOLS", // Partitioned by - "TOK_TABLEBUCKETS", // Clustered by - "TOK_TABLESKEWED", // Skewed by - "TOK_TABLEROWFORMAT", - "TOK_TABLESERIALIZER", - "TOK_FILEFORMAT_GENERIC", - "TOK_TABLEFILEFORMAT", // User-provided InputFormat and OutputFormat - "TOK_STORAGEHANDLER", // Storage handler - "TOK_TABLELOCATION", - "TOK_TABLEPROPERTIES"), - children) - val TableIdentifier(tblName, dbName) = extractTableIdent(tableNameParts) - - // TODO add bucket support - var tableDesc: HiveTable = HiveTable( - specifiedDatabase = dbName, - name = tblName, - schema = Seq.empty[HiveColumn], - partitionColumns = Seq.empty[HiveColumn], - properties = Map[String, String](), - serdeProperties = Map[String, String](), - tableType = if (externalTable.isDefined) ExternalTable else ManagedTable, - location = None, - inputFormat = None, - outputFormat = None, - serde = None, - viewText = None) - - // default storage type abbreviation (e.g. RCFile, ORC, PARQUET etc.) - val defaultStorageType = hiveConf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT) - // handle the default format for the storage type abbreviation - val hiveSerDe = HiveSerDe.sourceToSerDe(defaultStorageType, hiveConf).getOrElse { - HiveSerDe( - inputFormat = Option("org.apache.hadoop.mapred.TextInputFormat"), - outputFormat = Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) - } + nodeToColumns(cols, lowerCase = true).map(_.copy(hiveType = null)) + }.getOrElse(Seq.empty[HiveColumn]) - hiveSerDe.inputFormat.foreach(f => tableDesc = tableDesc.copy(inputFormat = Some(f))) - hiveSerDe.outputFormat.foreach(f => tableDesc = tableDesc.copy(outputFormat = Some(f))) - hiveSerDe.serde.foreach(f => tableDesc = tableDesc.copy(serde = Some(f))) + val properties = scala.collection.mutable.Map.empty[String, String] - children.collect { - case list @ Token("TOK_TABCOLLIST", _) => - val cols = SemanticAnalyzer.getColumns(list, true) - if (cols != null) { - tableDesc = tableDesc.copy( - schema = cols.asScala.map { field => - HiveColumn(field.getName, field.getType, field.getComment) - }) + maybeProperties.foreach { + case Token("TOK_TABLEPROPERTIES", list :: Nil) => + properties ++= getProperties(list) } - case Token("TOK_TABLECOMMENT", child :: Nil) => - val comment = SemanticAnalyzer.unescapeSQLString(child.getText) - // TODO support the sql text - tableDesc = tableDesc.copy(viewText = Option(comment)) - case Token("TOK_TABLEPARTCOLS", list @ Token("TOK_TABCOLLIST", _) :: Nil) => - val cols = SemanticAnalyzer.getColumns(list(0), false) - if (cols != null) { - tableDesc = tableDesc.copy( - partitionColumns = cols.asScala.map { field => - HiveColumn(field.getName, field.getType, field.getComment) - }) - } - case Token("TOK_TABLEROWFORMAT", Token("TOK_SERDEPROPS", child :: Nil) :: Nil) => - val serdeParams = new java.util.HashMap[String, String]() - child match { - case Token("TOK_TABLEROWFORMATFIELD", rowChild1 :: rowChild2) => - val fieldDelim = SemanticAnalyzer.unescapeSQLString (rowChild1.getText()) - serdeParams.put(serdeConstants.FIELD_DELIM, fieldDelim) - serdeParams.put(serdeConstants.SERIALIZATION_FORMAT, fieldDelim) - if (rowChild2.length > 1) { - val fieldEscape = SemanticAnalyzer.unescapeSQLString (rowChild2(0).getText) - serdeParams.put(serdeConstants.ESCAPE_CHAR, fieldEscape) - } - case Token("TOK_TABLEROWFORMATCOLLITEMS", rowChild :: Nil) => - val collItemDelim = SemanticAnalyzer.unescapeSQLString(rowChild.getText) - serdeParams.put(serdeConstants.COLLECTION_DELIM, collItemDelim) - case Token("TOK_TABLEROWFORMATMAPKEYS", rowChild :: Nil) => - val mapKeyDelim = SemanticAnalyzer.unescapeSQLString(rowChild.getText) - serdeParams.put(serdeConstants.MAPKEY_DELIM, mapKeyDelim) - case Token("TOK_TABLEROWFORMATLINES", rowChild :: Nil) => - val lineDelim = SemanticAnalyzer.unescapeSQLString(rowChild.getText) - if (!(lineDelim == "\n") && !(lineDelim == "10")) { - throw new AnalysisException( - SemanticAnalyzer.generateErrorMessage( - rowChild, - ErrorMsg.LINES_TERMINATED_BY_NON_NEWLINE.getMsg)) - } - serdeParams.put(serdeConstants.LINE_DELIM, lineDelim) - case Token("TOK_TABLEROWFORMATNULL", rowChild :: Nil) => - val nullFormat = SemanticAnalyzer.unescapeSQLString(rowChild.getText) - // TODO support the nullFormat - case _ => assert(false) - } - tableDesc = tableDesc.copy( - serdeProperties = tableDesc.serdeProperties ++ serdeParams.asScala) - case Token("TOK_TABLELOCATION", child :: Nil) => - var location = SemanticAnalyzer.unescapeSQLString(child.getText) - location = SemanticAnalyzer.relativeToAbsolutePath(hiveConf, location) - tableDesc = tableDesc.copy(location = Option(location)) - case Token("TOK_TABLESERIALIZER", child :: Nil) => - tableDesc = tableDesc.copy( - serde = Option(SemanticAnalyzer.unescapeSQLString(child.getChild(0).getText))) - if (child.getChildCount == 2) { - val serdeParams = new java.util.HashMap[String, String]() - SemanticAnalyzer.readProps( - (child.getChild(1).getChild(0)).asInstanceOf[ASTNode], serdeParams) - tableDesc = tableDesc.copy( - serdeProperties = tableDesc.serdeProperties ++ serdeParams.asScala) - } - case Token("TOK_FILEFORMAT_GENERIC", child :: Nil) => - child.getText().toLowerCase(Locale.ENGLISH) match { - case "orc" => - tableDesc = tableDesc.copy( - inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"), - outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")) - if (tableDesc.serde.isEmpty) { - tableDesc = tableDesc.copy( - serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) - } - - case "parquet" => - tableDesc = tableDesc.copy( - inputFormat = - Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), - outputFormat = - Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) - if (tableDesc.serde.isEmpty) { - tableDesc = tableDesc.copy( - serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) - } - - case "rcfile" => - tableDesc = tableDesc.copy( - inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"), - outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) - if (tableDesc.serde.isEmpty) { - tableDesc = tableDesc.copy( - serde = Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")) - } - case "textfile" => - tableDesc = tableDesc.copy( - inputFormat = - Option("org.apache.hadoop.mapred.TextInputFormat"), - outputFormat = - Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) - - case "sequencefile" => - tableDesc = tableDesc.copy( - inputFormat = Option("org.apache.hadoop.mapred.SequenceFileInputFormat"), - outputFormat = Option("org.apache.hadoop.mapred.SequenceFileOutputFormat")) - - case "avro" => - tableDesc = tableDesc.copy( - inputFormat = - Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"), - outputFormat = - Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat")) - if (tableDesc.serde.isEmpty) { - tableDesc = tableDesc.copy( - serde = Option("org.apache.hadoop.hive.serde2.avro.AvroSerDe")) + maybeComment.foreach { + case Token("TOK_TABLECOMMENT", child :: Nil) => + val comment = unescapeSQLString(child.text) + if (comment ne null) { + properties += ("comment" -> comment) } - - case _ => - throw new SemanticException( - s"Unrecognized file format in STORED AS clause: ${child.getText}") - } - - case Token("TOK_TABLESERIALIZER", - Token("TOK_SERDENAME", Token(serdeName, Nil) :: otherProps) :: Nil) => - tableDesc = tableDesc.copy(serde = Option(unquoteString(serdeName))) - - otherProps match { - case Token("TOK_TABLEPROPERTIES", list :: Nil) :: Nil => - tableDesc = tableDesc.copy( - serdeProperties = tableDesc.serdeProperties ++ getProperties(list)) - case Nil => } - case Token("TOK_TABLEPROPERTIES", list :: Nil) => - tableDesc = tableDesc.copy(properties = tableDesc.properties ++ getProperties(list)) - case list @ Token("TOK_TABLEFILEFORMAT", children) => - tableDesc = tableDesc.copy( - inputFormat = - Option(SemanticAnalyzer.unescapeSQLString(list.getChild(0).getText)), - outputFormat = - Option(SemanticAnalyzer.unescapeSQLString(list.getChild(1).getText))) - case Token("TOK_STORAGEHANDLER", _) => - throw new AnalysisException(ErrorMsg.CREATE_NON_NATIVE_AS.getMsg()) - case _ => // Unsupport features - } - - CreateTableAsSelect(tableDesc, nodeToPlan(query, context), allowExisting != None) - - // If its not a "CTAS" like above then take it 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", queryArgs) - if Seq("TOK_CTE", "TOK_FROM", "TOK_INSERT").contains(queryArgs.head.getText) => - - val (fromClause: Option[ASTNode], insertClauses, cteRelations) = - queryArgs match { - case Token("TOK_CTE", ctes) :: Token("TOK_FROM", from) :: inserts => - val cteRelations = ctes.map { node => - val relation = nodeToRelation(node, context).asInstanceOf[Subquery] - relation.alias -> relation - } - (Some(from.head), inserts, Some(cteRelations.toMap)) - case Token("TOK_FROM", from) :: inserts => - (Some(from.head), inserts, None) - case Token("TOK_INSERT", _) :: Nil => - (None, queryArgs, None) + createView(view, viewNameParts, query, schema, properties.toMap, + allowExisting.isDefined, replace.isDefined) } - // Return one query for each insert clause. - val queries = insertClauses.map { case Token("TOK_INSERT", singleInsert) => + case Token("TOK_CREATETABLE", children) + if children.collect { case t @ Token("TOK_QUERY", _) => t }.nonEmpty => + // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL val ( - intoClause :: - destClause :: - selectClause :: - selectDistinctClause :: - whereClause :: - groupByClause :: - rollupGroupByClause :: - cubeGroupByClause :: - groupingSetsClause :: - orderByClause :: - havingClause :: - sortByClause :: - clusterByClause :: - distributeByClause :: - limitClause :: - lateralViewClause :: - windowClause :: Nil) = { + Some(tableNameParts) :: + _ /* likeTable */ :: + externalTable :: + Some(query) :: + allowExisting +: + _) = getClauses( Seq( - "TOK_INSERT_INTO", - "TOK_DESTINATION", - "TOK_SELECT", - "TOK_SELECTDI", - "TOK_WHERE", - "TOK_GROUPBY", - "TOK_ROLLUP_GROUPBY", - "TOK_CUBE_GROUPBY", - "TOK_GROUPING_SETS", - "TOK_ORDERBY", - "TOK_HAVING", - "TOK_SORTBY", - "TOK_CLUSTERBY", - "TOK_DISTRIBUTEBY", - "TOK_LIMIT", - "TOK_LATERAL_VIEW", - "WINDOW"), - singleInsert) + "TOK_TABNAME", + "TOK_LIKETABLE", + "EXTERNAL", + "TOK_QUERY", + "TOK_IFNOTEXISTS", + "TOK_TABLECOMMENT", + "TOK_TABCOLLIST", + "TOK_TABLEPARTCOLS", // Partitioned by + "TOK_TABLEBUCKETS", // Clustered by + "TOK_TABLESKEWED", // Skewed by + "TOK_TABLEROWFORMAT", + "TOK_TABLESERIALIZER", + "TOK_FILEFORMAT_GENERIC", + "TOK_TABLEFILEFORMAT", // User-provided InputFormat and OutputFormat + "TOK_STORAGEHANDLER", // Storage handler + "TOK_TABLELOCATION", + "TOK_TABLEPROPERTIES"), + children) + val TableIdentifier(tblName, dbName) = extractTableIdent(tableNameParts) + + // TODO add bucket support + var tableDesc: HiveTable = HiveTable( + specifiedDatabase = dbName, + name = tblName, + schema = Seq.empty[HiveColumn], + partitionColumns = Seq.empty[HiveColumn], + properties = Map[String, String](), + serdeProperties = Map[String, String](), + tableType = if (externalTable.isDefined) ExternalTable else ManagedTable, + location = None, + inputFormat = None, + outputFormat = None, + serde = None, + viewText = None) + + // default storage type abbreviation (e.g. RCFile, ORC, PARQUET etc.) + val defaultStorageType = hiveConf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT) + // handle the default format for the storage type abbreviation + val hiveSerDe = HiveSerDe.sourceToSerDe(defaultStorageType, hiveConf).getOrElse { + HiveSerDe( + inputFormat = Option("org.apache.hadoop.mapred.TextInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) } - val relations = fromClause match { - case Some(f) => nodeToRelation(f, context) - case None => OneRowRelation - } + hiveSerDe.inputFormat.foreach(f => tableDesc = tableDesc.copy(inputFormat = Some(f))) + hiveSerDe.outputFormat.foreach(f => tableDesc = tableDesc.copy(outputFormat = Some(f))) + hiveSerDe.serde.foreach(f => tableDesc = tableDesc.copy(serde = Some(f))) - val withWhere = whereClause.map { whereNode => - val Seq(whereExpr) = whereNode.getChildren.asScala - Filter(nodeToExpr(whereExpr), relations) - }.getOrElse(relations) - - val select = - (selectClause orElse selectDistinctClause).getOrElse(sys.error("No select clause.")) - - // Script transformations are expressed as a select clause with a single expression of type - // TOK_TRANSFORM - val transformation = select.getChildren.iterator().next() match { - case Token("TOK_SELEXPR", - Token("TOK_TRANSFORM", - Token("TOK_EXPLIST", inputExprs) :: - Token("TOK_SERDE", inputSerdeClause) :: - Token("TOK_RECORDWRITER", writerClause) :: - // TODO: Need to support other types of (in/out)put - Token(script, Nil) :: - Token("TOK_SERDE", outputSerdeClause) :: - Token("TOK_RECORDREADER", readerClause) :: - outputClause) :: Nil) => - - val (output, schemaLess) = outputClause match { - case Token("TOK_ALIASLIST", aliases) :: Nil => - (aliases.map { case Token(name, Nil) => AttributeReference(name, StringType)() }, - false) - case Token("TOK_TABCOLLIST", attributes) :: Nil => - (attributes.map { case Token("TOK_TABCOL", Token(name, Nil) :: dataType :: Nil) => - AttributeReference(name, nodeToDataType(dataType))() }, false) - case Nil => - (List(AttributeReference("key", StringType)(), - AttributeReference("value", StringType)()), true) + children.collect { + case list @ Token("TOK_TABCOLLIST", _) => + val cols = nodeToColumns(list, lowerCase = true) + if (cols != null) { + tableDesc = tableDesc.copy(schema = cols) } - - type SerDeInfo = ( - Seq[(String, String)], // Input row format information - Option[String], // Optional input SerDe class - Seq[(String, String)], // Input SerDe properties - Boolean // Whether to use default record reader/writer - ) - - def matchSerDe(clause: Seq[ASTNode]): SerDeInfo = clause match { - case Token("TOK_SERDEPROPS", propsClause) :: Nil => - val rowFormat = propsClause.map { - case Token(name, Token(value, Nil) :: Nil) => (name, value) + case Token("TOK_TABLECOMMENT", child :: Nil) => + val comment = unescapeSQLString(child.text) + // TODO support the sql text + tableDesc = tableDesc.copy(viewText = Option(comment)) + case Token("TOK_TABLEPARTCOLS", list @ Token("TOK_TABCOLLIST", _) :: Nil) => + val cols = nodeToColumns(list.head, lowerCase = false) + if (cols != null) { + tableDesc = tableDesc.copy(partitionColumns = cols) + } + case Token("TOK_TABLEROWFORMAT", Token("TOK_SERDEPROPS", child :: Nil) :: Nil) => + val serdeParams = new java.util.HashMap[String, String]() + child match { + case Token("TOK_TABLEROWFORMATFIELD", rowChild1 :: rowChild2) => + val fieldDelim = unescapeSQLString (rowChild1.text) + serdeParams.put(serdeConstants.FIELD_DELIM, fieldDelim) + serdeParams.put(serdeConstants.SERIALIZATION_FORMAT, fieldDelim) + if (rowChild2.length > 1) { + val fieldEscape = unescapeSQLString (rowChild2.head.text) + serdeParams.put(serdeConstants.ESCAPE_CHAR, fieldEscape) } - (rowFormat, None, Nil, false) - - case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: Nil) :: Nil => - (Nil, Some(SemanticAnalyzer.unescapeSQLString(serdeClass)), Nil, false) - - case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: - Token("TOK_TABLEPROPERTIES", - Token("TOK_TABLEPROPLIST", propsClause) :: Nil) :: Nil) :: Nil => - val serdeProps = propsClause.map { - case Token("TOK_TABLEPROPERTY", Token(name, Nil) :: Token(value, Nil) :: Nil) => - (SemanticAnalyzer.unescapeSQLString(name), - SemanticAnalyzer.unescapeSQLString(value)) + case Token("TOK_TABLEROWFORMATCOLLITEMS", rowChild :: Nil) => + val collItemDelim = unescapeSQLString(rowChild.text) + serdeParams.put(serdeConstants.COLLECTION_DELIM, collItemDelim) + case Token("TOK_TABLEROWFORMATMAPKEYS", rowChild :: Nil) => + val mapKeyDelim = unescapeSQLString(rowChild.text) + serdeParams.put(serdeConstants.MAPKEY_DELIM, mapKeyDelim) + case Token("TOK_TABLEROWFORMATLINES", rowChild :: Nil) => + val lineDelim = unescapeSQLString(rowChild.text) + if (!(lineDelim == "\n") && !(lineDelim == "10")) { + throw new AnalysisException( + s"LINES TERMINATED BY only supports newline '\\n' right now: $rowChild") } - - // SPARK-10310: Special cases LazySimpleSerDe - // TODO Fully supports user-defined record reader/writer classes - val unescapedSerDeClass = SemanticAnalyzer.unescapeSQLString(serdeClass) - val useDefaultRecordReaderWriter = - unescapedSerDeClass == classOf[LazySimpleSerDe].getCanonicalName - (Nil, Some(unescapedSerDeClass), serdeProps, useDefaultRecordReaderWriter) - - case Nil => - // Uses default TextRecordReader/TextRecordWriter, sets field delimiter here - val serdeProps = Seq(serdeConstants.FIELD_DELIM -> "\t") - (Nil, Option(hiveConf.getVar(ConfVars.HIVESCRIPTSERDE)), serdeProps, true) - } - - val (inRowFormat, inSerdeClass, inSerdeProps, useDefaultRecordReader) = - matchSerDe(inputSerdeClause) - - val (outRowFormat, outSerdeClass, outSerdeProps, useDefaultRecordWriter) = - matchSerDe(outputSerdeClause) - - val unescapedScript = SemanticAnalyzer.unescapeSQLString(script) - - // TODO Adds support for user-defined record reader/writer classes - val recordReaderClass = if (useDefaultRecordReader) { - Option(hiveConf.getVar(ConfVars.HIVESCRIPTRECORDREADER)) - } else { - None + serdeParams.put(serdeConstants.LINE_DELIM, lineDelim) + case Token("TOK_TABLEROWFORMATNULL", rowChild :: Nil) => + val nullFormat = unescapeSQLString(rowChild.text) + // TODO support the nullFormat + case _ => assert(false) } - - val recordWriterClass = if (useDefaultRecordWriter) { - Option(hiveConf.getVar(ConfVars.HIVESCRIPTRECORDWRITER)) - } else { - None + tableDesc = tableDesc.copy( + serdeProperties = tableDesc.serdeProperties ++ serdeParams.asScala) + case Token("TOK_TABLELOCATION", child :: Nil) => + val location = EximUtil.relativeToAbsolutePath(hiveConf, unescapeSQLString(child.text)) + tableDesc = tableDesc.copy(location = Option(location)) + case Token("TOK_TABLESERIALIZER", child :: Nil) => + tableDesc = tableDesc.copy( + serde = Option(unescapeSQLString(child.children.head.text))) + if (child.numChildren == 2) { + // This is based on the readProps(..) method in + // ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java: + val serdeParams = child.children(1).children.head.children.map { + case Token(_, Token(prop, Nil) :: valueNode) => + val value = valueNode.headOption + .map(_.text) + .map(unescapeSQLString) + .orNull + (unescapeSQLString(prop), value) + }.toMap + tableDesc = tableDesc.copy(serdeProperties = tableDesc.serdeProperties ++ serdeParams) } + case Token("TOK_FILEFORMAT_GENERIC", child :: Nil) => + child.text.toLowerCase(Locale.ENGLISH) match { + case "orc" => + tableDesc = tableDesc.copy( + inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")) + if (tableDesc.serde.isEmpty) { + tableDesc = tableDesc.copy( + serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) + } - val schema = HiveScriptIOSchema( - inRowFormat, outRowFormat, - inSerdeClass, outSerdeClass, - inSerdeProps, outSerdeProps, - recordReaderClass, recordWriterClass, - schemaLess) - - Some( - logical.ScriptTransformation( - inputExprs.map(nodeToExpr), - unescapedScript, - output, - withWhere, schema)) - case _ => None - } - - val withLateralView = lateralViewClause.map { lv => - val Token("TOK_SELECT", - Token("TOK_SELEXPR", clauses) :: Nil) = lv.getChildren.iterator().next() - - val alias = getClause("TOK_TABALIAS", clauses).getChildren.iterator().next() - .asInstanceOf[ASTNode].getText - - val (generator, attributes) = nodesToGenerator(clauses) - Generate( - generator, - join = true, - outer = false, - Some(alias.toLowerCase), - attributes.map(UnresolvedAttribute(_)), - withWhere) - }.getOrElse(withWhere) - - // The projection of the query can either be a normal projection, an aggregation - // (if there is a group by) or a script transformation. - val withProject: LogicalPlan = transformation.getOrElse { - val selectExpressions = - select.getChildren.asScala.flatMap(selExprNodeToExpr).map(UnresolvedAlias(_)) - Seq( - groupByClause.map(e => e match { - case Token("TOK_GROUPBY", children) => - // Not a transformation so must be either project or aggregation. - Aggregate(children.map(nodeToExpr), selectExpressions, withLateralView) - case _ => sys.error("Expect GROUP BY") - }), - groupingSetsClause.map(e => e match { - case Token("TOK_GROUPING_SETS", children) => - val(groupByExprs, masks) = extractGroupingSet(children) - GroupingSets(masks, groupByExprs, withLateralView, selectExpressions) - case _ => sys.error("Expect GROUPING SETS") - }), - rollupGroupByClause.map(e => e match { - case Token("TOK_ROLLUP_GROUPBY", children) => - Aggregate(Seq(Rollup(children.map(nodeToExpr))), selectExpressions, withLateralView) - case _ => sys.error("Expect WITH ROLLUP") - }), - cubeGroupByClause.map(e => e match { - case Token("TOK_CUBE_GROUPBY", children) => - Aggregate(Seq(Cube(children.map(nodeToExpr))), selectExpressions, withLateralView) - case _ => sys.error("Expect WITH CUBE") - }), - Some(Project(selectExpressions, withLateralView))).flatten.head - } + case "parquet" => + tableDesc = tableDesc.copy( + inputFormat = + Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), + outputFormat = + Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) + if (tableDesc.serde.isEmpty) { + tableDesc = tableDesc.copy( + serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) + } - // Handle HAVING clause. - val withHaving = havingClause.map { h => - val havingExpr = h.getChildren.asScala match { case Seq(hexpr) => nodeToExpr(hexpr) } - // Note that we added a cast to boolean. If the expression itself is already boolean, - // the optimizer will get rid of the unnecessary cast. - Filter(Cast(havingExpr, BooleanType), withProject) - }.getOrElse(withProject) - - // Handle SELECT DISTINCT - val withDistinct = - if (selectDistinctClause.isDefined) Distinct(withHaving) else withHaving - - // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause. - val withSort = - (orderByClause, sortByClause, distributeByClause, clusterByClause) match { - case (Some(totalOrdering), None, None, None) => - Sort(totalOrdering.getChildren.asScala.map(nodeToSortOrder), true, withDistinct) - case (None, Some(perPartitionOrdering), None, None) => - Sort( - perPartitionOrdering.getChildren.asScala.map(nodeToSortOrder), - false, withDistinct) - case (None, None, Some(partitionExprs), None) => - RepartitionByExpression( - partitionExprs.getChildren.asScala.map(nodeToExpr), withDistinct) - case (None, Some(perPartitionOrdering), Some(partitionExprs), None) => - Sort( - perPartitionOrdering.getChildren.asScala.map(nodeToSortOrder), false, - RepartitionByExpression( - partitionExprs.getChildren.asScala.map(nodeToExpr), - withDistinct)) - case (None, None, None, Some(clusterExprs)) => - Sort( - clusterExprs.getChildren.asScala.map(nodeToExpr).map(SortOrder(_, Ascending)), - false, - RepartitionByExpression( - clusterExprs.getChildren.asScala.map(nodeToExpr), - withDistinct)) - case (None, None, None, None) => withDistinct - case _ => sys.error("Unsupported set of ordering / distribution clauses.") - } + case "rcfile" => + tableDesc = tableDesc.copy( + inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) + if (tableDesc.serde.isEmpty) { + tableDesc = tableDesc.copy(serde = + Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")) + } - val withLimit = - limitClause.map(l => nodeToExpr(l.getChildren.iterator().next())) - .map(Limit(_, withSort)) - .getOrElse(withSort) - - // Collect all window specifications defined in the WINDOW clause. - val windowDefinitions = windowClause.map(_.getChildren.asScala.collect { - case Token("TOK_WINDOWDEF", - Token(windowName, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => - windowName -> nodesToWindowSpecification(spec) - }.toMap) - // Handle cases like - // window w1 as (partition by p_mfgr order by p_name - // range between 2 preceding and 2 following), - // w2 as w1 - val resolvedCrossReference = windowDefinitions.map { - windowDefMap => windowDefMap.map { - case (windowName, WindowSpecReference(other)) => - (windowName, windowDefMap(other).asInstanceOf[WindowSpecDefinition]) - case o => o.asInstanceOf[(String, WindowSpecDefinition)] - } - } + case "textfile" => + tableDesc = tableDesc.copy( + inputFormat = + Option("org.apache.hadoop.mapred.TextInputFormat"), + outputFormat = + Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) - val withWindowDefinitions = - resolvedCrossReference.map(WithWindowDefinition(_, withLimit)).getOrElse(withLimit) - - // TOK_INSERT_INTO means to add files to the table. - // TOK_DESTINATION means to overwrite the table. - val resultDestination = - (intoClause orElse destClause).getOrElse(sys.error("No destination found.")) - val overwrite = intoClause.isEmpty - nodeToDest( - resultDestination, - withWindowDefinitions, - overwrite) - } + case "sequencefile" => + tableDesc = tableDesc.copy( + inputFormat = Option("org.apache.hadoop.mapred.SequenceFileInputFormat"), + outputFormat = Option("org.apache.hadoop.mapred.SequenceFileOutputFormat")) - // If there are multiple INSERTS just UNION them together into on query. - val query = queries.reduceLeft(Union) + case "avro" => + tableDesc = tableDesc.copy( + inputFormat = + Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"), + outputFormat = + Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat")) + if (tableDesc.serde.isEmpty) { + tableDesc = tableDesc.copy( + serde = Option("org.apache.hadoop.hive.serde2.avro.AvroSerDe")) + } - // return With plan if there is CTE - cteRelations.map(With(query, _)).getOrElse(query) + case _ => + throw new AnalysisException( + s"Unrecognized file format in STORED AS clause: ${child.text}") + } - // HIVE-9039 renamed TOK_UNION => TOK_UNIONALL while adding TOK_UNIONDISTINCT - case Token("TOK_UNIONALL", left :: right :: Nil) => - Union(nodeToPlan(left, context), nodeToPlan(right, context)) + case Token("TOK_TABLESERIALIZER", + Token("TOK_SERDENAME", Token(serdeName, Nil) :: otherProps) :: Nil) => + tableDesc = tableDesc.copy(serde = Option(unquoteString(serdeName))) - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for $node:\n ${dumpTree(a).toString} ") - } + otherProps match { + case Token("TOK_TABLEPROPERTIES", list :: Nil) :: Nil => + tableDesc = tableDesc.copy( + serdeProperties = tableDesc.serdeProperties ++ getProperties(list)) + case _ => + } - val allJoinTokens = "(TOK_.*JOIN)".r - val laterViewToken = "TOK_LATERAL_VIEW(.*)".r - def nodeToRelation(node: Node, context: Context): LogicalPlan = node match { - case Token("TOK_SUBQUERY", - query :: Token(alias, Nil) :: Nil) => - Subquery(cleanIdentifier(alias), nodeToPlan(query, context)) - - case Token(laterViewToken(isOuter), selectClause :: relationClause :: Nil) => - val Token("TOK_SELECT", - Token("TOK_SELEXPR", clauses) :: Nil) = selectClause - - val alias = getClause("TOK_TABALIAS", clauses).getChildren.iterator().next() - .asInstanceOf[ASTNode].getText - - val (generator, attributes) = nodesToGenerator(clauses) - Generate( - generator, - join = true, - outer = isOuter.nonEmpty, - Some(alias.toLowerCase), - attributes.map(UnresolvedAttribute(_)), - nodeToRelation(relationClause, context)) - - /* All relations, possibly with aliases or sampling clauses. */ - case Token("TOK_TABREF", clauses) => - // If the last clause is not a token then it's the alias of the table. - val (nonAliasClauses, aliasClause) = - if (clauses.last.getText.startsWith("TOK")) { - (clauses, None) - } else { - (clauses.dropRight(1), Some(clauses.last)) + case Token("TOK_TABLEPROPERTIES", list :: Nil) => + tableDesc = tableDesc.copy(properties = tableDesc.properties ++ getProperties(list)) + case list @ Token("TOK_TABLEFILEFORMAT", _) => + tableDesc = tableDesc.copy( + inputFormat = + Option(unescapeSQLString(list.children.head.text)), + outputFormat = + Option(unescapeSQLString(list.children(1).text))) + case Token("TOK_STORAGEHANDLER", _) => + throw new AnalysisException( + "CREATE TABLE AS SELECT cannot be used for a non-native table") + case _ => // Unsupport features } - val (Some(tableNameParts) :: - splitSampleClause :: - bucketSampleClause :: Nil) = { - getClauses(Seq("TOK_TABNAME", "TOK_TABLESPLITSAMPLE", "TOK_TABLEBUCKETSAMPLE"), - nonAliasClauses) - } - - val tableIdent = extractTableIdent(tableNameParts) - val alias = aliasClause.map { case Token(a, Nil) => cleanIdentifier(a) } - val relation = UnresolvedRelation(tableIdent, alias) - - // Apply sampling if requested. - (bucketSampleClause orElse splitSampleClause).map { - case Token("TOK_TABLESPLITSAMPLE", - Token("TOK_ROWCOUNT", Nil) :: - Token(count, Nil) :: Nil) => - Limit(Literal(count.toInt), relation) - case Token("TOK_TABLESPLITSAMPLE", - Token("TOK_PERCENT", Nil) :: - Token(fraction, Nil) :: Nil) => - // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling - // function takes X PERCENT as the input and the range of X is [0, 100], we need to - // adjust the fraction. - require( - fraction.toDouble >= (0.0 - RandomSampler.roundingEpsilon) - && fraction.toDouble <= (100.0 + RandomSampler.roundingEpsilon), - s"Sampling fraction ($fraction) must be on interval [0, 100]") - Sample(0.0, fraction.toDouble / 100, withReplacement = false, (math.random * 1000).toInt, - relation) - case Token("TOK_TABLEBUCKETSAMPLE", - Token(numerator, Nil) :: - Token(denominator, Nil) :: Nil) => - val fraction = numerator.toDouble / denominator.toDouble - Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, relation) - case a: ASTNode => - throw new NotImplementedError( - s"""No parse rules for sampling clause: ${a.getType}, text: ${a.getText} : - |${dumpTree(a).toString}" + - """.stripMargin) - }.getOrElse(relation) - - case Token("TOK_UNIQUEJOIN", joinArgs) => - val tableOrdinals = - joinArgs.zipWithIndex.filter { - case (arg, i) => arg.getText == "TOK_TABREF" - }.map(_._2) - - val isPreserved = tableOrdinals.map(i => (i - 1 < 0) || joinArgs(i - 1).getText == "PRESERVE") - val tables = tableOrdinals.map(i => nodeToRelation(joinArgs(i), context)) - val joinExpressions = - tableOrdinals.map(i => joinArgs(i + 1).getChildren.asScala.map(nodeToExpr)) - - val joinConditions = joinExpressions.sliding(2).map { - case Seq(c1, c2) => - val predicates = (c1, c2).zipped.map { case (e1, e2) => EqualTo(e1, e2): Expression } - predicates.reduceLeft(And) - }.toBuffer - - val joinType = isPreserved.sliding(2).map { - case Seq(true, true) => FullOuter - case Seq(true, false) => LeftOuter - case Seq(false, true) => RightOuter - case Seq(false, false) => Inner - }.toBuffer - - val joinedTables = tables.reduceLeft(Join(_, _, Inner, None)) - - // Must be transform down. - val joinedResult = joinedTables transform { - case j: Join => - j.copy( - condition = Some(joinConditions.remove(joinConditions.length - 1)), - joinType = joinType.remove(joinType.length - 1)) - } - - val groups = joinExpressions.head.indices.map(i => Coalesce(joinExpressions.map(_(i)))) - - // Unique join is not really the same as an outer join so we must group together results where - // the joinExpressions are the same, taking the First of each value is only okay because the - // user of a unique join is implicitly promising that there is only one result. - // TODO: This doesn't actually work since [[Star]] is not a valid aggregate expression. - // instead we should figure out how important supporting this feature is and whether it is - // worth the number of hacks that will be required to implement it. Namely, we need to add - // some sort of mapped star expansion that would expand all child output row to be similarly - // named output expressions where some aggregate expression has been applied (i.e. First). - // Aggregate(groups, Star(None, First(_)) :: Nil, joinedResult) - throw new UnsupportedOperationException - - case Token(allJoinTokens(joinToken), - relation1 :: - relation2 :: other) => - if (!(other.size <= 1)) { - sys.error(s"Unsupported join operation: $other") - } - - val joinType = joinToken match { - case "TOK_JOIN" => Inner - case "TOK_CROSSJOIN" => Inner - case "TOK_RIGHTOUTERJOIN" => RightOuter - case "TOK_LEFTOUTERJOIN" => LeftOuter - case "TOK_FULLOUTERJOIN" => FullOuter - case "TOK_LEFTSEMIJOIN" => LeftSemi - case "TOK_ANTIJOIN" => throw new NotImplementedError("Anti join not supported") - } - Join(nodeToRelation(relation1, context), - nodeToRelation(relation2, context), - joinType, - other.headOption.map(nodeToExpr)) - - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") - } + CreateTableAsSelect(tableDesc, nodeToPlan(query), allowExisting.isDefined) - def nodeToSortOrder(node: Node): SortOrder = node match { - case Token("TOK_TABSORTCOLNAMEASC", sortExpr :: Nil) => - SortOrder(nodeToExpr(sortExpr), Ascending) - case Token("TOK_TABSORTCOLNAMEDESC", sortExpr :: Nil) => - SortOrder(nodeToExpr(sortExpr), Descending) + // If its not a "CTAS" like above then take it as a native command + case Token("TOK_CREATETABLE", _) => + NativePlaceholder - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") - } + // Support "TRUNCATE TABLE table_name [PARTITION partition_spec]" + case Token("TOK_TRUNCATETABLE", Token("TOK_TABLE_PARTITION", table) :: Nil) => + NativePlaceholder - val destinationToken = "TOK_DESTINATION|TOK_INSERT_INTO".r - protected def nodeToDest( - node: Node, - query: LogicalPlan, - overwrite: Boolean): LogicalPlan = node match { - case Token(destinationToken(), - Token("TOK_DIR", - Token("TOK_TMP_FILE", Nil) :: Nil) :: Nil) => - query - - case Token(destinationToken(), - Token("TOK_TAB", - tableArgs) :: Nil) => - val Some(tableNameParts) :: partitionClause :: Nil = - getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) - - val tableIdent = extractTableIdent(tableNameParts) - - val partitionKeys = partitionClause.map(_.getChildren.asScala.map { - // Parse partitions. We also make keys case insensitive. - case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => - cleanIdentifier(key.toLowerCase) -> Some(PlanUtils.stripQuotes(value)) - case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => - cleanIdentifier(key.toLowerCase) -> None - }.toMap).getOrElse(Map.empty) - - InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, false) - - case Token(destinationToken(), - Token("TOK_TAB", - tableArgs) :: - Token("TOK_IFNOTEXISTS", - ifNotExists) :: Nil) => - val Some(tableNameParts) :: partitionClause :: Nil = - getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) - - val tableIdent = extractTableIdent(tableNameParts) - - val partitionKeys = partitionClause.map(_.getChildren.asScala.map { - // Parse partitions. We also make keys case insensitive. - case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => - cleanIdentifier(key.toLowerCase) -> Some(PlanUtils.stripQuotes(value)) - case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => - cleanIdentifier(key.toLowerCase) -> None - }.toMap).getOrElse(Map.empty) - - InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, true) - - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for ${a.getName}:" + - s"\n ${dumpTree(a).toString} ") + case _ => + super.nodeToPlan(node) + } } - protected def selExprNodeToExpr(node: Node): Option[Expression] = node match { - case Token("TOK_SELEXPR", e :: Nil) => - Some(nodeToExpr(e)) - - case Token("TOK_SELEXPR", e :: Token(alias, Nil) :: Nil) => - Some(Alias(nodeToExpr(e), cleanIdentifier(alias))()) - - case Token("TOK_SELEXPR", e :: aliasChildren) => - var aliasNames = ArrayBuffer[String]() - aliasChildren.foreach { _ match { - case Token(name, Nil) => aliasNames += cleanIdentifier(name) + protected override def nodeToDescribeFallback(node: ASTNode): LogicalPlan = NativePlaceholder + + protected override def nodeToTransformation( + node: ASTNode, + child: LogicalPlan): Option[ScriptTransformation] = node match { + case Token("TOK_SELEXPR", + Token("TOK_TRANSFORM", + Token("TOK_EXPLIST", inputExprs) :: + Token("TOK_SERDE", inputSerdeClause) :: + Token("TOK_RECORDWRITER", writerClause) :: + // TODO: Need to support other types of (in/out)put + Token(script, Nil) :: + Token("TOK_SERDE", outputSerdeClause) :: + Token("TOK_RECORDREADER", readerClause) :: + outputClause) :: Nil) => + + val (output, schemaLess) = outputClause match { + case Token("TOK_ALIASLIST", aliases) :: Nil => + (aliases.map { case Token(name, Nil) => AttributeReference(name, StringType)() }, + false) + case Token("TOK_TABCOLLIST", attributes) :: Nil => + (attributes.map { case Token("TOK_TABCOL", Token(name, Nil) :: dataType :: Nil) => + AttributeReference(name, nodeToDataType(dataType))() }, false) + case Nil => + (List(AttributeReference("key", StringType)(), + AttributeReference("value", StringType)()), true) case _ => - } + noParseRule("Transform", node) } - Some(MultiAlias(nodeToExpr(e), aliasNames)) - - /* Hints are ignored */ - case Token("TOK_HINTLIST", _) => None - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for ${a.getName }:" + - s"\n ${dumpTree(a).toString } ") - } - - protected val escapedIdentifier = "`([^`]+)`".r - protected val doubleQuotedString = "\"([^\"]+)\"".r - protected val singleQuotedString = "'([^']+)'".r + type SerDeInfo = ( + Seq[(String, String)], // Input row format information + Option[String], // Optional input SerDe class + Seq[(String, String)], // Input SerDe properties + Boolean // Whether to use default record reader/writer + ) + + def matchSerDe(clause: Seq[ASTNode]): SerDeInfo = clause match { + case Token("TOK_SERDEPROPS", propsClause) :: Nil => + val rowFormat = propsClause.map { + case Token(name, Token(value, Nil) :: Nil) => (name, value) + } + (rowFormat, None, Nil, false) - protected def unquoteString(str: String) = str match { - case singleQuotedString(s) => s - case doubleQuotedString(s) => s - case other => other - } + case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: Nil) :: Nil => + (Nil, Some(unescapeSQLString(serdeClass)), Nil, false) - /** Strips backticks from ident if present */ - protected def cleanIdentifier(ident: String): String = ident match { - case escapedIdentifier(i) => i - case plainIdent => plainIdent - } + case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: + Token("TOK_TABLEPROPERTIES", + Token("TOK_TABLEPROPLIST", propsClause) :: Nil) :: Nil) :: Nil => + val serdeProps = propsClause.map { + case Token("TOK_TABLEPROPERTY", Token(name, Nil) :: Token(value, Nil) :: Nil) => + (unescapeSQLString(name), unescapeSQLString(value)) + } - val numericAstTypes = Seq( - SparkSqlParser.Number, - SparkSqlParser.TinyintLiteral, - SparkSqlParser.SmallintLiteral, - SparkSqlParser.BigintLiteral, - SparkSqlParser.DecimalLiteral) - - /* Case insensitive matches */ - val COUNT = "(?i)COUNT".r - val SUM = "(?i)SUM".r - val AND = "(?i)AND".r - val OR = "(?i)OR".r - val NOT = "(?i)NOT".r - val TRUE = "(?i)TRUE".r - val FALSE = "(?i)FALSE".r - val LIKE = "(?i)LIKE".r - val RLIKE = "(?i)RLIKE".r - val REGEXP = "(?i)REGEXP".r - val IN = "(?i)IN".r - val DIV = "(?i)DIV".r - val BETWEEN = "(?i)BETWEEN".r - val WHEN = "(?i)WHEN".r - val CASE = "(?i)CASE".r - - protected def nodeToExpr(node: Node): Expression = node match { - /* Attribute References */ - case Token("TOK_TABLE_OR_COL", - Token(name, Nil) :: Nil) => - UnresolvedAttribute.quoted(cleanIdentifier(name)) - case Token(".", qualifier :: Token(attr, Nil) :: Nil) => - nodeToExpr(qualifier) match { - case UnresolvedAttribute(nameParts) => - UnresolvedAttribute(nameParts :+ cleanIdentifier(attr)) - case other => UnresolvedExtractValue(other, Literal(attr)) + // SPARK-10310: Special cases LazySimpleSerDe + // TODO Fully supports user-defined record reader/writer classes + val unescapedSerDeClass = unescapeSQLString(serdeClass) + val useDefaultRecordReaderWriter = + unescapedSerDeClass == classOf[LazySimpleSerDe].getCanonicalName + (Nil, Some(unescapedSerDeClass), serdeProps, useDefaultRecordReaderWriter) + + case Nil => + // Uses default TextRecordReader/TextRecordWriter, sets field delimiter here + val serdeProps = Seq(serdeConstants.FIELD_DELIM -> "\t") + (Nil, Option(hiveConf.getVar(ConfVars.HIVESCRIPTSERDE)), serdeProps, true) } - /* Stars (*) */ - case Token("TOK_ALLCOLREF", Nil) => UnresolvedStar(None) - // The format of dbName.tableName.* cannot be parsed by HiveParser. TOK_TABNAME will only - // has a single child which is tableName. - case Token("TOK_ALLCOLREF", Token("TOK_TABNAME", Token(name, Nil) :: Nil) :: Nil) => - UnresolvedStar(Some(UnresolvedAttribute.parseAttributeName(name))) - - /* Aggregate Functions */ - case Token("TOK_FUNCTIONDI", Token(COUNT(), Nil) :: args) => - Count(args.map(nodeToExpr)).toAggregateExpression(isDistinct = true) - case Token("TOK_FUNCTIONSTAR", Token(COUNT(), Nil) :: Nil) => - Count(Literal(1)).toAggregateExpression() - - /* Casts */ - case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), StringType) - 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) - case Token("TOK_FUNCTION", Token("TOK_BIGINT", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), LongType) - case Token("TOK_FUNCTION", Token("TOK_FLOAT", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), FloatType) - case Token("TOK_FUNCTION", Token("TOK_DOUBLE", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), DoubleType) - case Token("TOK_FUNCTION", Token("TOK_SMALLINT", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), ShortType) - case Token("TOK_FUNCTION", Token("TOK_TINYINT", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), ByteType) - case Token("TOK_FUNCTION", Token("TOK_BINARY", Nil) :: arg :: Nil) => - 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.USER_DEFAULT) - 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) => nodeToExpr(child) - 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)) - case Token("/", 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)) - 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)) - - /* 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)) - case Token(">=", left :: right:: Nil) => GreaterThanOrEqual(nodeToExpr(left), nodeToExpr(right)) - case Token("<", left :: right:: Nil) => LessThan(nodeToExpr(left), nodeToExpr(right)) - case Token("<=", left :: right:: Nil) => LessThanOrEqual(nodeToExpr(left), nodeToExpr(right)) - case Token(LIKE(), left :: right:: Nil) => Like(nodeToExpr(left), nodeToExpr(right)) - case Token(RLIKE(), left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) - case Token(REGEXP(), left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) - case Token("TOK_FUNCTION", Token("TOK_ISNOTNULL", Nil) :: child :: Nil) => - IsNotNull(nodeToExpr(child)) - case Token("TOK_FUNCTION", Token("TOK_ISNULL", Nil) :: child :: Nil) => - IsNull(nodeToExpr(child)) - case Token("TOK_FUNCTION", Token(IN(), Nil) :: value :: list) => - In(nodeToExpr(value), list.map(nodeToExpr)) - case Token("TOK_FUNCTION", - Token(BETWEEN(), Nil) :: - kw :: - target :: - minValue :: - maxValue :: Nil) => - - val targetExpression = nodeToExpr(target) - 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) - } + val (inRowFormat, inSerdeClass, inSerdeProps, useDefaultRecordReader) = + matchSerDe(inputSerdeClause) - /* Boolean Logic */ - case Token(AND(), left :: right:: Nil) => And(nodeToExpr(left), nodeToExpr(right)) - case Token(OR(), left :: right:: Nil) => Or(nodeToExpr(left), nodeToExpr(right)) - case Token(NOT(), child :: Nil) => Not(nodeToExpr(child)) - case Token("!", child :: Nil) => Not(nodeToExpr(child)) - - /* Case statements */ - case Token("TOK_FUNCTION", Token(WHEN(), Nil) :: branches) => - CaseWhen(branches.map(nodeToExpr)) - case Token("TOK_FUNCTION", Token(CASE(), Nil) :: branches) => - val keyExpr = nodeToExpr(branches.head) - CaseKeyWhen(keyExpr, branches.drop(1).map(nodeToExpr)) - - /* Complex datatype manipulation */ - case Token("[", child :: ordinal :: Nil) => - UnresolvedExtractValue(nodeToExpr(child), nodeToExpr(ordinal)) - - /* Window Functions */ - case Token(name, args :+ Token("TOK_WINDOWSPEC", spec)) => - val function = nodeToExpr(Token(name, args)) - nodesToWindowSpecification(spec) match { - case reference: WindowSpecReference => - UnresolvedWindowExpression(function, reference) - case definition: WindowSpecDefinition => - WindowExpression(function, definition) - } + val (outRowFormat, outSerdeClass, outSerdeProps, useDefaultRecordWriter) = + matchSerDe(outputSerdeClause) - /* UDFs - Must be last otherwise will preempt built in functions */ - case Token("TOK_FUNCTION", Token(name, Nil) :: args) => - UnresolvedFunction(name, args.map(nodeToExpr), isDistinct = false) - // Aggregate function with DISTINCT keyword. - case Token("TOK_FUNCTIONDI", Token(name, Nil) :: args) => - UnresolvedFunction(name, args.map(nodeToExpr), isDistinct = true) - case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: args) => - UnresolvedFunction(name, UnresolvedStar(None) :: Nil, isDistinct = false) - - /* Literals */ - case Token("TOK_NULL", Nil) => Literal.create(null, NullType) - case Token(TRUE(), Nil) => Literal.create(true, BooleanType) - case Token(FALSE(), Nil) => Literal.create(false, BooleanType) - case Token("TOK_STRINGLITERALSEQUENCE", strings) => - Literal(strings.map(s => SemanticAnalyzer.unescapeSQLString(s.getText)).mkString) - - // This code is adapted from - // /ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java#L223 - case ast: ASTNode if numericAstTypes contains ast.getType => - var v: Literal = null - try { - if (ast.getText.endsWith("L")) { - // Literal bigint. - v = Literal.create(ast.getText.substring(0, ast.getText.length() - 1).toLong, LongType) - } else if (ast.getText.endsWith("S")) { - // Literal smallint. - v = Literal.create(ast.getText.substring(0, ast.getText.length() - 1).toShort, ShortType) - } else if (ast.getText.endsWith("Y")) { - // Literal tinyint. - v = Literal.create(ast.getText.substring(0, ast.getText.length() - 1).toByte, ByteType) - } else if (ast.getText.endsWith("BD") || ast.getText.endsWith("D")) { - // Literal decimal - val strVal = ast.getText.stripSuffix("D").stripSuffix("B") - v = Literal(Decimal(strVal)) - } else { - v = Literal.create(ast.getText.toDouble, DoubleType) - v = Literal.create(ast.getText.toLong, LongType) - v = Literal.create(ast.getText.toInt, IntegerType) - } - } catch { - case nfe: NumberFormatException => // Do nothing - } + val unescapedScript = unescapeSQLString(script) - if (v == null) { - sys.error(s"Failed to parse number '${ast.getText}'.") + // TODO Adds support for user-defined record reader/writer classes + val recordReaderClass = if (useDefaultRecordReader) { + Option(hiveConf.getVar(ConfVars.HIVESCRIPTRECORDREADER)) } else { - v + None } - case ast: ASTNode if ast.getType == SparkSqlParser.StringLiteral => - Literal(SemanticAnalyzer.unescapeSQLString(ast.getText)) - - case ast: ASTNode if ast.getType == SparkSqlParser.TOK_DATELITERAL => - Literal(Date.valueOf(ast.getText.substring(1, ast.getText.length - 1))) - - case ast: ASTNode if ast.getType == SparkSqlParser.TOK_CHARSETLITERAL => - Literal(SemanticAnalyzer.charSetString(ast.getChild(0).getText, ast.getChild(1).getText)) - - case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_YEAR_MONTH_LITERAL => - Literal(CalendarInterval.fromYearMonthString(ast.getText)) - - case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_DAY_TIME_LITERAL => - Literal(CalendarInterval.fromDayTimeString(ast.getText)) - - case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_YEAR_LITERAL => - Literal(CalendarInterval.fromSingleUnitString("year", ast.getText)) - - case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_MONTH_LITERAL => - Literal(CalendarInterval.fromSingleUnitString("month", ast.getText)) - - case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_DAY_LITERAL => - Literal(CalendarInterval.fromSingleUnitString("day", ast.getText)) - - case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_HOUR_LITERAL => - Literal(CalendarInterval.fromSingleUnitString("hour", ast.getText)) - - case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_MINUTE_LITERAL => - Literal(CalendarInterval.fromSingleUnitString("minute", ast.getText)) - - case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_SECOND_LITERAL => - Literal(CalendarInterval.fromSingleUnitString("second", ast.getText)) - - case a: ASTNode => - throw new NotImplementedError( - s"""No parse rules for ASTNode type: ${a.getType}, text: ${a.getText} : - |${dumpTree(a).toString}" + - """.stripMargin) - } - - /* Case insensitive matches for Window Specification */ - val PRECEDING = "(?i)preceding".r - val FOLLOWING = "(?i)following".r - val CURRENT = "(?i)current".r - def nodesToWindowSpecification(nodes: Seq[ASTNode]): WindowSpec = nodes match { - case Token(windowName, Nil) :: Nil => - // Refer to a window spec defined in the window clause. - WindowSpecReference(windowName) - case Nil => - // OVER() - WindowSpecDefinition( - partitionSpec = Nil, - orderSpec = Nil, - frameSpecification = UnspecifiedFrame) - case spec => - val (partitionClause :: rowFrame :: rangeFrame :: Nil) = - getClauses( - Seq( - "TOK_PARTITIONINGSPEC", - "TOK_WINDOWRANGE", - "TOK_WINDOWVALUES"), - spec) - - // Handle Partition By and Order By. - val (partitionSpec, orderSpec) = partitionClause.map { partitionAndOrdering => - val (partitionByClause :: orderByClause :: sortByClause :: clusterByClause :: Nil) = - getClauses( - Seq("TOK_DISTRIBUTEBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_CLUSTERBY"), - partitionAndOrdering.getChildren.asScala.asInstanceOf[Seq[ASTNode]]) - - (partitionByClause, orderByClause.orElse(sortByClause), clusterByClause) match { - case (Some(partitionByExpr), Some(orderByExpr), None) => - (partitionByExpr.getChildren.asScala.map(nodeToExpr), - orderByExpr.getChildren.asScala.map(nodeToSortOrder)) - case (Some(partitionByExpr), None, None) => - (partitionByExpr.getChildren.asScala.map(nodeToExpr), Nil) - case (None, Some(orderByExpr), None) => - (Nil, orderByExpr.getChildren.asScala.map(nodeToSortOrder)) - case (None, None, Some(clusterByExpr)) => - val expressions = clusterByExpr.getChildren.asScala.map(nodeToExpr) - (expressions, expressions.map(SortOrder(_, Ascending))) - case _ => - throw new NotImplementedError( - s"""No parse rules for Node ${partitionAndOrdering.getName} - """.stripMargin) - } - }.getOrElse { - (Nil, Nil) + val recordWriterClass = if (useDefaultRecordWriter) { + Option(hiveConf.getVar(ConfVars.HIVESCRIPTRECORDWRITER)) + } else { + None } - // Handle Window Frame - val windowFrame = - if (rowFrame.isEmpty && rangeFrame.isEmpty) { - UnspecifiedFrame - } else { - val frameType = rowFrame.map(_ => RowFrame).getOrElse(RangeFrame) - def nodeToBoundary(node: Node): FrameBoundary = node match { - case Token(PRECEDING(), Token(count, Nil) :: Nil) => - if (count.toLowerCase() == "unbounded") { - UnboundedPreceding - } else { - ValuePreceding(count.toInt) - } - case Token(FOLLOWING(), Token(count, Nil) :: Nil) => - if (count.toLowerCase() == "unbounded") { - UnboundedFollowing - } else { - ValueFollowing(count.toInt) - } - case Token(CURRENT(), Nil) => CurrentRow - case _ => - throw new NotImplementedError( - s"""No parse rules for the Window Frame Boundary based on Node ${node.getName} - """.stripMargin) - } - - rowFrame.orElse(rangeFrame).map { frame => - frame.getChildren.asScala.toList match { - case precedingNode :: followingNode :: Nil => - SpecifiedWindowFrame( - frameType, - nodeToBoundary(precedingNode), - nodeToBoundary(followingNode)) - case precedingNode :: Nil => - SpecifiedWindowFrame(frameType, nodeToBoundary(precedingNode), CurrentRow) - case _ => - throw new NotImplementedError( - s"""No parse rules for the Window Frame based on Node ${frame.getName} - """.stripMargin) - } - }.getOrElse(sys.error(s"If you see this, please file a bug report with your query.")) - } - - WindowSpecDefinition(partitionSpec, orderSpec, windowFrame) + val schema = HiveScriptIOSchema( + inRowFormat, outRowFormat, + inSerdeClass, outSerdeClass, + inSerdeProps, outSerdeProps, + recordReaderClass, recordWriterClass, + schemaLess) + + Some( + ScriptTransformation( + inputExprs.map(nodeToExpr), + unescapedScript, + output, + child, schema)) + case _ => None } - val explode = "(?i)explode".r - val jsonTuple = "(?i)json_tuple".r - def nodesToGenerator(nodes: Seq[Node]): (Generator, Seq[String]) = { - val function = nodes.head - - val attributes = nodes.flatMap { - case Token(a, Nil) => a.toLowerCase :: Nil - case _ => Nil - } - - function match { - case Token("TOK_FUNCTION", Token(explode(), Nil) :: child :: Nil) => - (Explode(nodeToExpr(child)), attributes) - - case Token("TOK_FUNCTION", Token(jsonTuple(), Nil) :: children) => - (JsonTuple(children.map(nodeToExpr)), attributes) - - case Token("TOK_FUNCTION", Token(functionName, Nil) :: children) => - val functionInfo: FunctionInfo = - Option(FunctionRegistry.getFunctionInfo(functionName.toLowerCase)).getOrElse( - sys.error(s"Couldn't find function $functionName")) - val functionClassName = functionInfo.getFunctionClass.getName - - (HiveGenericUDTF( - new HiveFunctionWrapper(functionClassName), - children.map(nodeToExpr)), attributes) - - case a: ASTNode => - throw new NotImplementedError( - s"""No parse rules for ASTNode type: ${a.getType}, text: ${a.getText}, tree: - |${dumpTree(a).toString} - """.stripMargin) - } + protected override def nodeToGenerator(node: ASTNode): Generator = node match { + case Token("TOK_FUNCTION", Token(functionName, Nil) :: children) => + val functionInfo: FunctionInfo = + Option(FunctionRegistry.getFunctionInfo(functionName.toLowerCase)).getOrElse( + sys.error(s"Couldn't find function $functionName")) + val functionClassName = functionInfo.getFunctionClass.getName + HiveGenericUDTF(new HiveFunctionWrapper(functionClassName), children.map(nodeToExpr)) + case other => super.nodeToGenerator(node) } - def dumpTree(node: Node, builder: StringBuilder = new StringBuilder, indent: Int = 0) - : StringBuilder = { - node match { - case a: ASTNode => builder.append( - (" " * indent) + a.getText + " " + - a.getLine + ", " + - a.getTokenStartIndex + "," + - a.getTokenStopIndex + ", " + - a.getCharPositionInLine + "\n") - case other => sys.error(s"Non ASTNode encountered: $other") + // This is based the getColumns methods in + // ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java + protected def nodeToColumns(node: ASTNode, lowerCase: Boolean): Seq[HiveColumn] = { + node.children.map(_.children).collect { + case Token(rawColName, Nil) :: colTypeNode :: comment => + val colName = if (!lowerCase) rawColName + else rawColName.toLowerCase + HiveColumn( + cleanIdentifier(colName), + nodeToTypeString(colTypeNode), + comment.headOption.map(n => unescapeSQLString(n.text)).orNull) } + } - Option(node.getChildren).map(_.asScala).getOrElse(Nil).foreach(dumpTree(_, builder, indent + 1)) - builder + // This is based on the following methods in + // ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java: + // getTypeStringFromAST + // getStructTypeStringFromAST + // getUnionTypeStringFromAST + protected def nodeToTypeString(node: ASTNode): String = node.tokenType match { + case SparkSqlParser.TOK_LIST => + val listType :: Nil = node.children + val listTypeString = nodeToTypeString(listType) + s"${serdeConstants.LIST_TYPE_NAME}<$listTypeString>" + + case SparkSqlParser.TOK_MAP => + val keyType :: valueType :: Nil = node.children + val keyTypeString = nodeToTypeString(keyType) + val valueTypeString = nodeToTypeString(valueType) + s"${serdeConstants.MAP_TYPE_NAME}<$keyTypeString,$valueTypeString>" + + case SparkSqlParser.TOK_STRUCT => + val typeNode = node.children.head + require(typeNode.children.nonEmpty, "Struct must have one or more columns.") + val structColStrings = typeNode.children.map { columnNode => + val Token(colName, Nil) :: colTypeNode :: Nil = columnNode.children + cleanIdentifier(colName) + ":" + nodeToTypeString(colTypeNode) + } + s"${serdeConstants.STRUCT_TYPE_NAME}<${structColStrings.mkString(",")}>" + + case SparkSqlParser.TOK_UNIONTYPE => + val typeNode = node.children.head + val unionTypesString = typeNode.children.map(nodeToTypeString).mkString(",") + s"${serdeConstants.UNION_TYPE_NAME}<$unionTypesString>" + + case SparkSqlParser.TOK_CHAR => + val Token(size, Nil) :: Nil = node.children + s"${serdeConstants.CHAR_TYPE_NAME}($size)" + + case SparkSqlParser.TOK_VARCHAR => + val Token(size, Nil) :: Nil = node.children + s"${serdeConstants.VARCHAR_TYPE_NAME}($size)" + + case SparkSqlParser.TOK_DECIMAL => + val precisionAndScale = node.children match { + case Token(precision, Nil) :: Token(scale, Nil) :: Nil => + precision + "," + scale + case Token(precision, Nil) :: Nil => + precision + "," + HiveDecimal.USER_DEFAULT_SCALE + case Nil => + HiveDecimal.USER_DEFAULT_PRECISION + "," + HiveDecimal.USER_DEFAULT_SCALE + case _ => + noParseRule("Decimal", node) + } + s"${serdeConstants.DECIMAL_TYPE_NAME}($precisionAndScale)" + + // Simple data types. + case SparkSqlParser.TOK_BOOLEAN => serdeConstants.BOOLEAN_TYPE_NAME + case SparkSqlParser.TOK_TINYINT => serdeConstants.TINYINT_TYPE_NAME + case SparkSqlParser.TOK_SMALLINT => serdeConstants.SMALLINT_TYPE_NAME + case SparkSqlParser.TOK_INT => serdeConstants.INT_TYPE_NAME + case SparkSqlParser.TOK_BIGINT => serdeConstants.BIGINT_TYPE_NAME + case SparkSqlParser.TOK_FLOAT => serdeConstants.FLOAT_TYPE_NAME + case SparkSqlParser.TOK_DOUBLE => serdeConstants.DOUBLE_TYPE_NAME + case SparkSqlParser.TOK_STRING => serdeConstants.STRING_TYPE_NAME + case SparkSqlParser.TOK_BINARY => serdeConstants.BINARY_TYPE_NAME + case SparkSqlParser.TOK_DATE => serdeConstants.DATE_TYPE_NAME + case SparkSqlParser.TOK_TIMESTAMP => serdeConstants.TIMESTAMP_TYPE_NAME + case SparkSqlParser.TOK_INTERVAL_YEAR_MONTH => serdeConstants.INTERVAL_YEAR_MONTH_TYPE_NAME + case SparkSqlParser.TOK_INTERVAL_DAY_TIME => serdeConstants.INTERVAL_DAY_TIME_TYPE_NAME + case SparkSqlParser.TOK_DATETIME => serdeConstants.DATETIME_TYPE_NAME + case _ => null } + } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index 400f7f3708cf4..a2d283622ca52 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -21,6 +21,7 @@ import scala.util.Try import org.scalatest.BeforeAndAfter +import org.apache.spark.sql.catalyst.parser.ParseDriver import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.hive.test.TestHiveSingleton @@ -116,8 +117,9 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd * @param token a unique token in the string that should be indicated by the exception */ def positionTest(name: String, query: String, token: String): Unit = { + def ast = ParseDriver.parse(query, hiveContext.conf) def parseTree = - Try(quietly(HiveQl.dumpTree(HiveQl.getAst(query)))).getOrElse("") + Try(quietly(ast.treeString)).getOrElse("") test(name) { val error = intercept[AnalysisException] { @@ -139,10 +141,7 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd val expectedStart = line.indexOf(token) val actualStart = error.startPosition.getOrElse { - fail( - s"start not returned for error on token $token\n" + - HiveQl.dumpTree(HiveQl.getAst(query)) - ) + fail(s"start not returned for error on token $token\n${ast.treeString}") } assert(expectedStart === actualStart, s"""Incorrect start position. From fcd013cf70e7890aa25a8fe3cb6c8b36bf0e1f04 Mon Sep 17 00:00:00 2001 From: zero323 Date: Wed, 6 Jan 2016 11:58:33 -0800 Subject: [PATCH 006/110] [SPARK-12006][ML][PYTHON] Fix GMM failure if initialModel is not None If initial model passed to GMM is not empty it causes `net.razorvine.pickle.PickleException`. It can be fixed by converting `initialModel.weights` to `list`. Author: zero323 Closes #9986 from zero323/SPARK-12006. --- python/pyspark/mllib/clustering.py | 2 +- python/pyspark/mllib/tests.py | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index c9e6f1dec6bf8..48daa87e82d13 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -346,7 +346,7 @@ def train(cls, rdd, k, convergenceTol=1e-3, maxIterations=100, seed=None, initia if initialModel.k != k: raise Exception("Mismatched cluster count, initialModel.k = %s, however k = %s" % (initialModel.k, k)) - initialModelWeights = initialModel.weights + initialModelWeights = list(initialModel.weights) initialModelMu = [initialModel.gaussians[i].mu for i in range(initialModel.k)] initialModelSigma = [initialModel.gaussians[i].sigma for i in range(initialModel.k)] java_model = callMLlibFunc("trainGaussianMixtureModel", rdd.map(_convert_to_vector), diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 6ed03e35828ed..97fed7662ea90 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -475,6 +475,18 @@ def test_gmm_deterministic(self): for c1, c2 in zip(clusters1.weights, clusters2.weights): self.assertEqual(round(c1, 7), round(c2, 7)) + def test_gmm_with_initial_model(self): + from pyspark.mllib.clustering import GaussianMixture + data = self.sc.parallelize([ + (-10, -5), (-9, -4), (10, 5), (9, 4) + ]) + + gmm1 = GaussianMixture.train(data, 2, convergenceTol=0.001, + maxIterations=10, seed=63) + gmm2 = GaussianMixture.train(data, 2, convergenceTol=0.001, + maxIterations=10, seed=63, initialModel=gmm1) + self.assertAlmostEqual((gmm1.weights - gmm2.weights).sum(), 0.0) + def test_classification(self): from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes from pyspark.mllib.tree import DecisionTree, DecisionTreeModel, RandomForest,\ From f82ebb15224ec5375f25f67d598ec3ef1cb65210 Mon Sep 17 00:00:00 2001 From: BenFradet Date: Wed, 6 Jan 2016 12:01:05 -0800 Subject: [PATCH 007/110] [SPARK-12368][ML][DOC] Better doc for the binary classification evaluator' metricName For the BinaryClassificationEvaluator, the scaladoc doesn't mention that "areaUnderPR" is supported, only that the default is "areadUnderROC". Also, in the documentation, it is said that: "The default metric used to choose the best ParamMap can be overriden by the setMetric method in each of these evaluators." However, the method is called setMetricName. This PR aims to fix both issues. Author: BenFradet Closes #10328 from BenFradet/SPARK-12368. --- docs/ml-guide.md | 4 ++-- .../spark/ml/evaluation/BinaryClassificationEvaluator.scala | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 44a316a07dfef..1343753bce246 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -628,7 +628,7 @@ Currently, `spark.ml` supports model selection using the [`CrossValidator`](api/ The `Evaluator` can be a [`RegressionEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.RegressionEvaluator) for regression problems, a [`BinaryClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.BinaryClassificationEvaluator) for binary data, or a [`MultiClassClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.MultiClassClassificationEvaluator) -for multiclass problems. The default metric used to choose the best `ParamMap` can be overriden by the `setMetric` +for multiclass problems. The default metric used to choose the best `ParamMap` can be overriden by the `setMetricName` method in each of these evaluators. The `ParamMap` which produces the best evaluation metric (averaged over the `$k$` folds) is selected as the best model. @@ -951,4 +951,4 @@ model.transform(test) {% endhighlight %} - \ No newline at end of file + diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index bfb70963b151d..f71726f110e84 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -39,8 +39,7 @@ class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override va def this() = this(Identifiable.randomUID("binEval")) /** - * param for metric name in evaluation - * Default: areaUnderROC + * param for metric name in evaluation (supports `"areaUnderROC"` (default), `"areaUnderPR"`) * @group param */ @Since("1.2.0") From 1e6648d62fb82b708ea54c51cd23bfe4f542856e Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 6 Jan 2016 12:03:01 -0800 Subject: [PATCH 008/110] [SPARK-12617][PYSPARK] Move Py4jCallbackConnectionCleaner to Streaming Move Py4jCallbackConnectionCleaner to Streaming because the callback server starts only in StreamingContext. Author: Shixiong Zhu Closes #10621 from zsxwing/SPARK-12617-2. --- python/pyspark/context.py | 61 ---------------------------- python/pyspark/streaming/context.py | 63 +++++++++++++++++++++++++++++ 2 files changed, 63 insertions(+), 61 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 5e4aeac330c5a..529d16b480399 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -54,64 +54,6 @@ } -class Py4jCallbackConnectionCleaner(object): - - """ - A cleaner to clean up callback connections that are not closed by Py4j. See SPARK-12617. - It will scan all callback connections every 30 seconds and close the dead connections. - """ - - def __init__(self, gateway): - self._gateway = gateway - self._stopped = False - self._timer = None - self._lock = RLock() - - def start(self): - if self._stopped: - return - - def clean_closed_connections(): - from py4j.java_gateway import quiet_close, quiet_shutdown - - callback_server = self._gateway._callback_server - with callback_server.lock: - try: - closed_connections = [] - for connection in callback_server.connections: - if not connection.isAlive(): - quiet_close(connection.input) - quiet_shutdown(connection.socket) - quiet_close(connection.socket) - closed_connections.append(connection) - - for closed_connection in closed_connections: - callback_server.connections.remove(closed_connection) - except Exception: - import traceback - traceback.print_exc() - - self._start_timer(clean_closed_connections) - - self._start_timer(clean_closed_connections) - - def _start_timer(self, f): - from threading import Timer - - with self._lock: - if not self._stopped: - self._timer = Timer(30.0, f) - self._timer.daemon = True - self._timer.start() - - def stop(self): - with self._lock: - self._stopped = True - if self._timer: - self._timer.cancel() - self._timer = None - - class SparkContext(object): """ @@ -126,7 +68,6 @@ class SparkContext(object): _active_spark_context = None _lock = RLock() _python_includes = None # zip and egg files that need to be added to PYTHONPATH - _py4j_cleaner = None PACKAGE_EXTENSIONS = ('.zip', '.egg', '.jar') @@ -303,8 +244,6 @@ def _ensure_initialized(cls, instance=None, gateway=None): if not SparkContext._gateway: SparkContext._gateway = gateway or launch_gateway() SparkContext._jvm = SparkContext._gateway.jvm - _py4j_cleaner = Py4jCallbackConnectionCleaner(SparkContext._gateway) - _py4j_cleaner.start() if instance: if (SparkContext._active_spark_context and diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index 5cc4bbde39958..0f1f005ce3edf 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -19,6 +19,7 @@ import os import sys +from threading import RLock, Timer from py4j.java_gateway import java_import, JavaObject @@ -32,6 +33,63 @@ __all__ = ["StreamingContext"] +class Py4jCallbackConnectionCleaner(object): + + """ + A cleaner to clean up callback connections that are not closed by Py4j. See SPARK-12617. + It will scan all callback connections every 30 seconds and close the dead connections. + """ + + def __init__(self, gateway): + self._gateway = gateway + self._stopped = False + self._timer = None + self._lock = RLock() + + def start(self): + if self._stopped: + return + + def clean_closed_connections(): + from py4j.java_gateway import quiet_close, quiet_shutdown + + callback_server = self._gateway._callback_server + if callback_server: + with callback_server.lock: + try: + closed_connections = [] + for connection in callback_server.connections: + if not connection.isAlive(): + quiet_close(connection.input) + quiet_shutdown(connection.socket) + quiet_close(connection.socket) + closed_connections.append(connection) + + for closed_connection in closed_connections: + callback_server.connections.remove(closed_connection) + except Exception: + import traceback + traceback.print_exc() + + self._start_timer(clean_closed_connections) + + self._start_timer(clean_closed_connections) + + def _start_timer(self, f): + with self._lock: + if not self._stopped: + self._timer = Timer(30.0, f) + self._timer.daemon = True + self._timer.start() + + def stop(self): + with self._lock: + self._stopped = True + if self._timer: + self._timer.cancel() + self._timer = None + + class StreamingContext(object): """ Main entry point for Spark Streaming functionality. A StreamingContext @@ -47,6 +105,9 @@ class StreamingContext(object): # Reference to a currently active StreamingContext _activeContext = None + # A cleaner to clean leak sockets of callback server every 30 seconds + _py4j_cleaner = None + def __init__(self, sparkContext, batchDuration=None, jssc=None): """ Create a new StreamingContext. @@ -95,6 +156,8 @@ def _ensure_initialized(cls): jgws = JavaObject("GATEWAY_SERVER", gw._gateway_client) # update the port of CallbackClient with real port gw.jvm.PythonDStream.updatePythonGatewayPort(jgws, gw._python_proxy_port) + _py4j_cleaner = Py4jCallbackConnectionCleaner(gw) + _py4j_cleaner.start() # register serializer for TransformFunction # it happens before creating SparkContext when loading from checkpointing From 19e4e9febf9bb4fd69f6d7bc13a54844e4e096f1 Mon Sep 17 00:00:00 2001 From: huangzhaowei Date: Wed, 6 Jan 2016 12:48:57 -0800 Subject: [PATCH 009/110] [SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url. Author: huangzhaowei Closes #10617 from SaintBacchus/SPARK-12672. --- .../org/apache/spark/streaming/scheduler/JobScheduler.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 1ed6fb0aa9d52..2c57706636fa5 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 @@ -26,7 +26,8 @@ import org.apache.spark.Logging import org.apache.spark.rdd.PairRDDFunctions import org.apache.spark.streaming._ import org.apache.spark.streaming.ui.UIUtils -import org.apache.spark.util.{EventLoop, ThreadUtils, Utils} +import org.apache.spark.ui.{UIUtils => SparkUIUtils} +import org.apache.spark.util.{EventLoop, ThreadUtils} private[scheduler] sealed trait JobSchedulerEvent @@ -203,7 +204,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { try { val formattedTime = UIUtils.formatBatchTime( job.time.milliseconds, ssc.graph.batchDuration.milliseconds, showYYYYMMSS = false) - val batchUrl = s"/streaming/batch/?id=${job.time.milliseconds}" + val batchUrl = s"${SparkUIUtils.uiRoot}/streaming/batch/?id=${job.time.milliseconds}" val batchLinkText = s"[output operation ${job.outputOpId}, batch time ${formattedTime}]" ssc.sc.setJobDescription( From cbaea9591f089171f3af654d1f9a52916e9f28b9 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 6 Jan 2016 13:51:50 -0800 Subject: [PATCH 010/110] Revert "[SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url." This reverts commit 19e4e9febf9bb4fd69f6d7bc13a54844e4e096f1. Will merge #10618 instead. --- .../org/apache/spark/streaming/scheduler/JobScheduler.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) 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 2c57706636fa5..1ed6fb0aa9d52 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 @@ -26,8 +26,7 @@ import org.apache.spark.Logging import org.apache.spark.rdd.PairRDDFunctions import org.apache.spark.streaming._ import org.apache.spark.streaming.ui.UIUtils -import org.apache.spark.ui.{UIUtils => SparkUIUtils} -import org.apache.spark.util.{EventLoop, ThreadUtils} +import org.apache.spark.util.{EventLoop, ThreadUtils, Utils} private[scheduler] sealed trait JobSchedulerEvent @@ -204,7 +203,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { try { val formattedTime = UIUtils.formatBatchTime( job.time.milliseconds, ssc.graph.batchDuration.milliseconds, showYYYYMMSS = false) - val batchUrl = s"${SparkUIUtils.uiRoot}/streaming/batch/?id=${job.time.milliseconds}" + val batchUrl = s"/streaming/batch/?id=${job.time.milliseconds}" val batchLinkText = s"[output operation ${job.outputOpId}, batch time ${formattedTime}]" ssc.sc.setJobDescription( From 6f7ba6409a39fd2e34865e3e7a84a3dd0b00d6a4 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 6 Jan 2016 15:54:00 -0800 Subject: [PATCH 011/110] [SPARK-12681] [SQL] split IdentifiersParser.g into two files To avoid to have a huge Java source (over 64K loc), that can't be compiled. cc hvanhovell Author: Davies Liu Closes #10624 from davies/split_ident. --- .../sql/catalyst/parser/ExpressionParser.g | 565 ++++++++++++++++++ .../sql/catalyst/parser/IdentifiersParser.g | 515 ---------------- .../sql/catalyst/parser/SparkSqlParser.g | 2 +- 3 files changed, 566 insertions(+), 516 deletions(-) create mode 100644 sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g new file mode 100644 index 0000000000000..cad770122d150 --- /dev/null +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g @@ -0,0 +1,565 @@ +/** + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT 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 file is an adaptation of Hive's org/apache/hadoop/hive/ql/IdentifiersParser.g grammar. +*/ + +parser grammar ExpressionParser; + +options +{ +output=AST; +ASTLabelType=CommonTree; +backtrack=false; +k=3; +} + +@members { + @Override + public Object recoverFromMismatchedSet(IntStream input, + RecognitionException re, BitSet follow) throws RecognitionException { + throw re; + } + @Override + public void displayRecognitionError(String[] tokenNames, + RecognitionException e) { + gParent.displayRecognitionError(tokenNames, e); + } + protected boolean useSQL11ReservedKeywordsForIdentifier() { + return gParent.useSQL11ReservedKeywordsForIdentifier(); + } +} + +@rulecatch { +catch (RecognitionException e) { + throw e; +} +} + +// fun(par1, par2, par3) +function +@init { gParent.pushMsg("function specification", state); } +@after { gParent.popMsg(state); } + : + functionName + LPAREN + ( + (STAR) => (star=STAR) + | (dist=KW_DISTINCT)? (selectExpression (COMMA selectExpression)*)? + ) + RPAREN (KW_OVER ws=window_specification)? + -> {$star != null}? ^(TOK_FUNCTIONSTAR functionName $ws?) + -> {$dist == null}? ^(TOK_FUNCTION functionName (selectExpression+)? $ws?) + -> ^(TOK_FUNCTIONDI functionName (selectExpression+)? $ws?) + ; + +functionName +@init { gParent.pushMsg("function name", state); } +@after { gParent.popMsg(state); } + : // Keyword IF is also a function name + (KW_IF | KW_ARRAY | KW_MAP | KW_STRUCT | KW_UNIONTYPE) => (KW_IF | KW_ARRAY | KW_MAP | KW_STRUCT | KW_UNIONTYPE) + | + (functionIdentifier) => functionIdentifier + | + {!useSQL11ReservedKeywordsForIdentifier()}? sql11ReservedKeywordsUsedAsCastFunctionName -> Identifier[$sql11ReservedKeywordsUsedAsCastFunctionName.text] + ; + +castExpression +@init { gParent.pushMsg("cast expression", state); } +@after { gParent.popMsg(state); } + : + KW_CAST + LPAREN + expression + KW_AS + primitiveType + RPAREN -> ^(TOK_FUNCTION primitiveType expression) + ; + +caseExpression +@init { gParent.pushMsg("case expression", state); } +@after { gParent.popMsg(state); } + : + KW_CASE expression + (KW_WHEN expression KW_THEN expression)+ + (KW_ELSE expression)? + KW_END -> ^(TOK_FUNCTION KW_CASE expression*) + ; + +whenExpression +@init { gParent.pushMsg("case expression", state); } +@after { gParent.popMsg(state); } + : + KW_CASE + ( KW_WHEN expression KW_THEN expression)+ + (KW_ELSE expression)? + KW_END -> ^(TOK_FUNCTION KW_WHEN expression*) + ; + +constant +@init { gParent.pushMsg("constant", state); } +@after { gParent.popMsg(state); } + : + Number + | dateLiteral + | timestampLiteral + | intervalLiteral + | StringLiteral + | stringLiteralSequence + | BigintLiteral + | SmallintLiteral + | TinyintLiteral + | DecimalLiteral + | charSetStringLiteral + | booleanValue + ; + +stringLiteralSequence + : + StringLiteral StringLiteral+ -> ^(TOK_STRINGLITERALSEQUENCE StringLiteral StringLiteral+) + ; + +charSetStringLiteral +@init { gParent.pushMsg("character string literal", state); } +@after { gParent.popMsg(state); } + : + csName=CharSetName csLiteral=CharSetLiteral -> ^(TOK_CHARSETLITERAL $csName $csLiteral) + ; + +dateLiteral + : + KW_DATE StringLiteral -> + { + // Create DateLiteral token, but with the text of the string value + // This makes the dateLiteral more consistent with the other type literals. + adaptor.create(TOK_DATELITERAL, $StringLiteral.text) + } + | + KW_CURRENT_DATE -> ^(TOK_FUNCTION KW_CURRENT_DATE) + ; + +timestampLiteral + : + KW_TIMESTAMP StringLiteral -> + { + adaptor.create(TOK_TIMESTAMPLITERAL, $StringLiteral.text) + } + | + KW_CURRENT_TIMESTAMP -> ^(TOK_FUNCTION KW_CURRENT_TIMESTAMP) + ; + +intervalLiteral + : + KW_INTERVAL StringLiteral qualifiers=intervalQualifiers -> + { + adaptor.create($qualifiers.tree.token.getType(), $StringLiteral.text) + } + ; + +intervalQualifiers + : + KW_YEAR KW_TO KW_MONTH -> TOK_INTERVAL_YEAR_MONTH_LITERAL + | KW_DAY KW_TO KW_SECOND -> TOK_INTERVAL_DAY_TIME_LITERAL + | KW_YEAR -> TOK_INTERVAL_YEAR_LITERAL + | KW_MONTH -> TOK_INTERVAL_MONTH_LITERAL + | KW_DAY -> TOK_INTERVAL_DAY_LITERAL + | KW_HOUR -> TOK_INTERVAL_HOUR_LITERAL + | KW_MINUTE -> TOK_INTERVAL_MINUTE_LITERAL + | KW_SECOND -> TOK_INTERVAL_SECOND_LITERAL + ; + +expression +@init { gParent.pushMsg("expression specification", state); } +@after { gParent.popMsg(state); } + : + precedenceOrExpression + ; + +atomExpression + : + (KW_NULL) => KW_NULL -> TOK_NULL + | (constant) => constant + | castExpression + | caseExpression + | whenExpression + | (functionName LPAREN) => function + | tableOrColumn + | LPAREN! expression RPAREN! + ; + + +precedenceFieldExpression + : + atomExpression ((LSQUARE^ expression RSQUARE!) | (DOT^ identifier))* + ; + +precedenceUnaryOperator + : + PLUS | MINUS | TILDE + ; + +nullCondition + : + KW_NULL -> ^(TOK_ISNULL) + | KW_NOT KW_NULL -> ^(TOK_ISNOTNULL) + ; + +precedenceUnaryPrefixExpression + : + (precedenceUnaryOperator^)* precedenceFieldExpression + ; + +precedenceUnarySuffixExpression + : precedenceUnaryPrefixExpression (a=KW_IS nullCondition)? + -> {$a != null}? ^(TOK_FUNCTION nullCondition precedenceUnaryPrefixExpression) + -> precedenceUnaryPrefixExpression + ; + + +precedenceBitwiseXorOperator + : + BITWISEXOR + ; + +precedenceBitwiseXorExpression + : + precedenceUnarySuffixExpression (precedenceBitwiseXorOperator^ precedenceUnarySuffixExpression)* + ; + + +precedenceStarOperator + : + STAR | DIVIDE | MOD | DIV + ; + +precedenceStarExpression + : + precedenceBitwiseXorExpression (precedenceStarOperator^ precedenceBitwiseXorExpression)* + ; + + +precedencePlusOperator + : + PLUS | MINUS + ; + +precedencePlusExpression + : + precedenceStarExpression (precedencePlusOperator^ precedenceStarExpression)* + ; + + +precedenceAmpersandOperator + : + AMPERSAND + ; + +precedenceAmpersandExpression + : + precedencePlusExpression (precedenceAmpersandOperator^ precedencePlusExpression)* + ; + + +precedenceBitwiseOrOperator + : + BITWISEOR + ; + +precedenceBitwiseOrExpression + : + precedenceAmpersandExpression (precedenceBitwiseOrOperator^ precedenceAmpersandExpression)* + ; + + +// Equal operators supporting NOT prefix +precedenceEqualNegatableOperator + : + KW_LIKE | KW_RLIKE | KW_REGEXP + ; + +precedenceEqualOperator + : + precedenceEqualNegatableOperator | EQUAL | EQUAL_NS | NOTEQUAL | LESSTHANOREQUALTO | LESSTHAN | GREATERTHANOREQUALTO | GREATERTHAN + ; + +subQueryExpression + : + LPAREN! selectStatement[true] RPAREN! + ; + +precedenceEqualExpression + : + (LPAREN precedenceBitwiseOrExpression COMMA) => precedenceEqualExpressionMutiple + | + precedenceEqualExpressionSingle + ; + +precedenceEqualExpressionSingle + : + (left=precedenceBitwiseOrExpression -> $left) + ( + (KW_NOT precedenceEqualNegatableOperator notExpr=precedenceBitwiseOrExpression) + -> ^(KW_NOT ^(precedenceEqualNegatableOperator $precedenceEqualExpressionSingle $notExpr)) + | (precedenceEqualOperator equalExpr=precedenceBitwiseOrExpression) + -> ^(precedenceEqualOperator $precedenceEqualExpressionSingle $equalExpr) + | (KW_NOT KW_IN LPAREN KW_SELECT)=> (KW_NOT KW_IN subQueryExpression) + -> ^(KW_NOT ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_IN) subQueryExpression $precedenceEqualExpressionSingle)) + | (KW_NOT KW_IN expressions) + -> ^(KW_NOT ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionSingle expressions)) + | (KW_IN LPAREN KW_SELECT)=> (KW_IN subQueryExpression) + -> ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_IN) subQueryExpression $precedenceEqualExpressionSingle) + | (KW_IN expressions) + -> ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionSingle expressions) + | ( KW_NOT KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) ) + -> ^(TOK_FUNCTION Identifier["between"] KW_TRUE $left $min $max) + | ( KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) ) + -> ^(TOK_FUNCTION Identifier["between"] KW_FALSE $left $min $max) + )* + | (KW_EXISTS LPAREN KW_SELECT)=> (KW_EXISTS subQueryExpression) -> ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_EXISTS) subQueryExpression) + ; + +expressions + : + LPAREN expression (COMMA expression)* RPAREN -> expression+ + ; + +//we transform the (col0, col1) in ((v00,v01),(v10,v11)) into struct(col0, col1) in (struct(v00,v01),struct(v10,v11)) +precedenceEqualExpressionMutiple + : + (LPAREN precedenceBitwiseOrExpression (COMMA precedenceBitwiseOrExpression)+ RPAREN -> ^(TOK_FUNCTION Identifier["struct"] precedenceBitwiseOrExpression+)) + ( (KW_IN LPAREN expressionsToStruct (COMMA expressionsToStruct)+ RPAREN) + -> ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionMutiple expressionsToStruct+) + | (KW_NOT KW_IN LPAREN expressionsToStruct (COMMA expressionsToStruct)+ RPAREN) + -> ^(KW_NOT ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionMutiple expressionsToStruct+))) + ; + +expressionsToStruct + : + LPAREN expression (COMMA expression)* RPAREN -> ^(TOK_FUNCTION Identifier["struct"] expression+) + ; + +precedenceNotOperator + : + KW_NOT + ; + +precedenceNotExpression + : + (precedenceNotOperator^)* precedenceEqualExpression + ; + + +precedenceAndOperator + : + KW_AND + ; + +precedenceAndExpression + : + precedenceNotExpression (precedenceAndOperator^ precedenceNotExpression)* + ; + + +precedenceOrOperator + : + KW_OR + ; + +precedenceOrExpression + : + precedenceAndExpression (precedenceOrOperator^ precedenceAndExpression)* + ; + + +booleanValue + : + KW_TRUE^ | KW_FALSE^ + ; + +booleanValueTok + : + KW_TRUE -> TOK_TRUE + | KW_FALSE -> TOK_FALSE + ; + +tableOrPartition + : + tableName partitionSpec? -> ^(TOK_TAB tableName partitionSpec?) + ; + +partitionSpec + : + KW_PARTITION + LPAREN partitionVal (COMMA partitionVal )* RPAREN -> ^(TOK_PARTSPEC partitionVal +) + ; + +partitionVal + : + identifier (EQUAL constant)? -> ^(TOK_PARTVAL identifier constant?) + ; + +dropPartitionSpec + : + KW_PARTITION + LPAREN dropPartitionVal (COMMA dropPartitionVal )* RPAREN -> ^(TOK_PARTSPEC dropPartitionVal +) + ; + +dropPartitionVal + : + identifier dropPartitionOperator constant -> ^(TOK_PARTVAL identifier dropPartitionOperator constant) + ; + +dropPartitionOperator + : + EQUAL | NOTEQUAL | LESSTHANOREQUALTO | LESSTHAN | GREATERTHANOREQUALTO | GREATERTHAN + ; + +sysFuncNames + : + KW_AND + | KW_OR + | KW_NOT + | KW_LIKE + | KW_IF + | KW_CASE + | KW_WHEN + | KW_TINYINT + | KW_SMALLINT + | KW_INT + | KW_BIGINT + | KW_FLOAT + | KW_DOUBLE + | KW_BOOLEAN + | KW_STRING + | KW_BINARY + | KW_ARRAY + | KW_MAP + | KW_STRUCT + | KW_UNIONTYPE + | EQUAL + | EQUAL_NS + | NOTEQUAL + | LESSTHANOREQUALTO + | LESSTHAN + | GREATERTHANOREQUALTO + | GREATERTHAN + | DIVIDE + | PLUS + | MINUS + | STAR + | MOD + | DIV + | AMPERSAND + | TILDE + | BITWISEOR + | BITWISEXOR + | KW_RLIKE + | KW_REGEXP + | KW_IN + | KW_BETWEEN + ; + +descFuncNames + : + (sysFuncNames) => sysFuncNames + | StringLiteral + | functionIdentifier + ; + +identifier + : + Identifier + | nonReserved -> Identifier[$nonReserved.text] + // If it decides to support SQL11 reserved keywords, i.e., useSQL11ReservedKeywordsForIdentifier()=false, + // the sql11keywords in existing q tests will NOT be added back. + | {useSQL11ReservedKeywordsForIdentifier()}? sql11ReservedKeywordsUsedAsIdentifier -> Identifier[$sql11ReservedKeywordsUsedAsIdentifier.text] + ; + +functionIdentifier +@init { gParent.pushMsg("function identifier", state); } +@after { gParent.popMsg(state); } + : db=identifier DOT fn=identifier + -> Identifier[$db.text + "." + $fn.text] + | + identifier + ; + +principalIdentifier +@init { gParent.pushMsg("identifier for principal spec", state); } +@after { gParent.popMsg(state); } + : identifier + | QuotedIdentifier + ; + +//The new version of nonReserved + sql11ReservedKeywordsUsedAsIdentifier = old version of nonReserved +//Non reserved keywords are basically the keywords that can be used as identifiers. +//All the KW_* are automatically not only keywords, but also reserved keywords. +//That means, they can NOT be used as identifiers. +//If you would like to use them as identifiers, put them in the nonReserved list below. +//If you are not sure, please refer to the SQL2011 column in +//http://www.postgresql.org/docs/9.5/static/sql-keywords-appendix.html +nonReserved + : + KW_ADD | KW_ADMIN | KW_AFTER | KW_ANALYZE | KW_ARCHIVE | KW_ASC | KW_BEFORE | KW_BUCKET | KW_BUCKETS + | KW_CASCADE | KW_CHANGE | KW_CLUSTER | KW_CLUSTERED | KW_CLUSTERSTATUS | KW_COLLECTION | KW_COLUMNS + | KW_COMMENT | KW_COMPACT | KW_COMPACTIONS | KW_COMPUTE | KW_CONCATENATE | KW_CONTINUE | KW_DATA | KW_DAY + | KW_DATABASES | KW_DATETIME | KW_DBPROPERTIES | KW_DEFERRED | KW_DEFINED | KW_DELIMITED | KW_DEPENDENCY + | KW_DESC | KW_DIRECTORIES | KW_DIRECTORY | KW_DISABLE | KW_DISTRIBUTE | KW_ELEM_TYPE + | KW_ENABLE | KW_ESCAPED | KW_EXCLUSIVE | KW_EXPLAIN | KW_EXPORT | KW_FIELDS | KW_FILE | KW_FILEFORMAT + | KW_FIRST | KW_FORMAT | KW_FORMATTED | KW_FUNCTIONS | KW_HOLD_DDLTIME | KW_HOUR | KW_IDXPROPERTIES | KW_IGNORE + | KW_INDEX | KW_INDEXES | KW_INPATH | KW_INPUTDRIVER | KW_INPUTFORMAT | KW_ITEMS | KW_JAR + | KW_KEYS | KW_KEY_TYPE | KW_LIMIT | KW_LINES | KW_LOAD | KW_LOCATION | KW_LOCK | KW_LOCKS | KW_LOGICAL | KW_LONG + | KW_MAPJOIN | KW_MATERIALIZED | KW_METADATA | KW_MINUS | KW_MINUTE | KW_MONTH | KW_MSCK | KW_NOSCAN | KW_NO_DROP | KW_OFFLINE + | KW_OPTION | KW_OUTPUTDRIVER | KW_OUTPUTFORMAT | KW_OVERWRITE | KW_OWNER | KW_PARTITIONED | KW_PARTITIONS | KW_PLUS | KW_PRETTY + | KW_PRINCIPALS | KW_PROTECTION | KW_PURGE | KW_READ | KW_READONLY | KW_REBUILD | KW_RECORDREADER | KW_RECORDWRITER + | KW_RELOAD | KW_RENAME | KW_REPAIR | KW_REPLACE | KW_REPLICATION | KW_RESTRICT | KW_REWRITE + | KW_ROLE | KW_ROLES | KW_SCHEMA | KW_SCHEMAS | KW_SECOND | KW_SEMI | KW_SERDE | KW_SERDEPROPERTIES | KW_SERVER | KW_SETS | KW_SHARED + | KW_SHOW | KW_SHOW_DATABASE | KW_SKEWED | KW_SORT | KW_SORTED | KW_SSL | KW_STATISTICS | KW_STORED + | KW_STREAMTABLE | KW_STRING | KW_STRUCT | KW_TABLES | KW_TBLPROPERTIES | KW_TEMPORARY | KW_TERMINATED + | KW_TINYINT | KW_TOUCH | KW_TRANSACTIONS | KW_UNARCHIVE | KW_UNDO | KW_UNIONTYPE | KW_UNLOCK | KW_UNSET + | KW_UNSIGNED | KW_URI | KW_USE | KW_UTC | KW_UTCTIMESTAMP | KW_VALUE_TYPE | KW_VIEW | KW_WHILE | KW_YEAR + | KW_WORK + | KW_TRANSACTION + | KW_WRITE + | KW_ISOLATION + | KW_LEVEL + | KW_SNAPSHOT + | KW_AUTOCOMMIT + | KW_ANTI +; + +//The following SQL2011 reserved keywords are used as cast function name only, but not as identifiers. +sql11ReservedKeywordsUsedAsCastFunctionName + : + KW_BIGINT | KW_BINARY | KW_BOOLEAN | KW_CURRENT_DATE | KW_CURRENT_TIMESTAMP | KW_DATE | KW_DOUBLE | KW_FLOAT | KW_INT | KW_SMALLINT | KW_TIMESTAMP + ; + +//The following SQL2011 reserved keywords are used as identifiers in many q tests, they may be added back due to backward compatibility. +//We are planning to remove the following whole list after several releases. +//Thus, please do not change the following list unless you know what to do. +sql11ReservedKeywordsUsedAsIdentifier + : + KW_ALL | KW_ALTER | KW_ARRAY | KW_AS | KW_AUTHORIZATION | KW_BETWEEN | KW_BIGINT | KW_BINARY | KW_BOOLEAN + | KW_BOTH | KW_BY | KW_CREATE | KW_CUBE | KW_CURRENT_DATE | KW_CURRENT_TIMESTAMP | KW_CURSOR | KW_DATE | KW_DECIMAL | KW_DELETE | KW_DESCRIBE + | KW_DOUBLE | KW_DROP | KW_EXISTS | KW_EXTERNAL | KW_FALSE | KW_FETCH | KW_FLOAT | KW_FOR | KW_FULL | KW_GRANT + | KW_GROUP | KW_GROUPING | KW_IMPORT | KW_IN | KW_INNER | KW_INSERT | KW_INT | KW_INTERSECT | KW_INTO | KW_IS | KW_LATERAL + | KW_LEFT | KW_LIKE | KW_LOCAL | KW_NONE | KW_NULL | KW_OF | KW_ORDER | KW_OUT | KW_OUTER | KW_PARTITION + | KW_PERCENT | KW_PROCEDURE | KW_RANGE | KW_READS | KW_REVOKE | KW_RIGHT + | KW_ROLLUP | KW_ROW | KW_ROWS | KW_SET | KW_SMALLINT | KW_TABLE | KW_TIMESTAMP | KW_TO | KW_TRIGGER | KW_TRUE + | KW_TRUNCATE | KW_UNION | KW_UPDATE | KW_USER | KW_USING | KW_VALUES | KW_WITH +//The following two keywords come from MySQL. Although they are not keywords in SQL2011, they are reserved keywords in MySQL. + | KW_REGEXP | KW_RLIKE + ; diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g index 86c6bd610f912..916eb6a7ac26b 100644 --- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g @@ -182,518 +182,3 @@ sortByClause columnRefOrderNotInParenthese -> ^(TOK_SORTBY columnRefOrderNotInParenthese) ) ; - -// fun(par1, par2, par3) -function -@init { gParent.pushMsg("function specification", state); } -@after { gParent.popMsg(state); } - : - functionName - LPAREN - ( - (STAR) => (star=STAR) - | (dist=KW_DISTINCT)? (selectExpression (COMMA selectExpression)*)? - ) - RPAREN (KW_OVER ws=window_specification)? - -> {$star != null}? ^(TOK_FUNCTIONSTAR functionName $ws?) - -> {$dist == null}? ^(TOK_FUNCTION functionName (selectExpression+)? $ws?) - -> ^(TOK_FUNCTIONDI functionName (selectExpression+)? $ws?) - ; - -functionName -@init { gParent.pushMsg("function name", state); } -@after { gParent.popMsg(state); } - : // Keyword IF is also a function name - (KW_IF | KW_ARRAY | KW_MAP | KW_STRUCT | KW_UNIONTYPE) => (KW_IF | KW_ARRAY | KW_MAP | KW_STRUCT | KW_UNIONTYPE) - | - (functionIdentifier) => functionIdentifier - | - {!useSQL11ReservedKeywordsForIdentifier()}? sql11ReservedKeywordsUsedAsCastFunctionName -> Identifier[$sql11ReservedKeywordsUsedAsCastFunctionName.text] - ; - -castExpression -@init { gParent.pushMsg("cast expression", state); } -@after { gParent.popMsg(state); } - : - KW_CAST - LPAREN - expression - KW_AS - primitiveType - RPAREN -> ^(TOK_FUNCTION primitiveType expression) - ; - -caseExpression -@init { gParent.pushMsg("case expression", state); } -@after { gParent.popMsg(state); } - : - KW_CASE expression - (KW_WHEN expression KW_THEN expression)+ - (KW_ELSE expression)? - KW_END -> ^(TOK_FUNCTION KW_CASE expression*) - ; - -whenExpression -@init { gParent.pushMsg("case expression", state); } -@after { gParent.popMsg(state); } - : - KW_CASE - ( KW_WHEN expression KW_THEN expression)+ - (KW_ELSE expression)? - KW_END -> ^(TOK_FUNCTION KW_WHEN expression*) - ; - -constant -@init { gParent.pushMsg("constant", state); } -@after { gParent.popMsg(state); } - : - Number - | dateLiteral - | timestampLiteral - | intervalLiteral - | StringLiteral - | stringLiteralSequence - | BigintLiteral - | SmallintLiteral - | TinyintLiteral - | DecimalLiteral - | charSetStringLiteral - | booleanValue - ; - -stringLiteralSequence - : - StringLiteral StringLiteral+ -> ^(TOK_STRINGLITERALSEQUENCE StringLiteral StringLiteral+) - ; - -charSetStringLiteral -@init { gParent.pushMsg("character string literal", state); } -@after { gParent.popMsg(state); } - : - csName=CharSetName csLiteral=CharSetLiteral -> ^(TOK_CHARSETLITERAL $csName $csLiteral) - ; - -dateLiteral - : - KW_DATE StringLiteral -> - { - // Create DateLiteral token, but with the text of the string value - // This makes the dateLiteral more consistent with the other type literals. - adaptor.create(TOK_DATELITERAL, $StringLiteral.text) - } - | - KW_CURRENT_DATE -> ^(TOK_FUNCTION KW_CURRENT_DATE) - ; - -timestampLiteral - : - KW_TIMESTAMP StringLiteral -> - { - adaptor.create(TOK_TIMESTAMPLITERAL, $StringLiteral.text) - } - | - KW_CURRENT_TIMESTAMP -> ^(TOK_FUNCTION KW_CURRENT_TIMESTAMP) - ; - -intervalLiteral - : - KW_INTERVAL StringLiteral qualifiers=intervalQualifiers -> - { - adaptor.create($qualifiers.tree.token.getType(), $StringLiteral.text) - } - ; - -intervalQualifiers - : - KW_YEAR KW_TO KW_MONTH -> TOK_INTERVAL_YEAR_MONTH_LITERAL - | KW_DAY KW_TO KW_SECOND -> TOK_INTERVAL_DAY_TIME_LITERAL - | KW_YEAR -> TOK_INTERVAL_YEAR_LITERAL - | KW_MONTH -> TOK_INTERVAL_MONTH_LITERAL - | KW_DAY -> TOK_INTERVAL_DAY_LITERAL - | KW_HOUR -> TOK_INTERVAL_HOUR_LITERAL - | KW_MINUTE -> TOK_INTERVAL_MINUTE_LITERAL - | KW_SECOND -> TOK_INTERVAL_SECOND_LITERAL - ; - -expression -@init { gParent.pushMsg("expression specification", state); } -@after { gParent.popMsg(state); } - : - precedenceOrExpression - ; - -atomExpression - : - (KW_NULL) => KW_NULL -> TOK_NULL - | (constant) => constant - | castExpression - | caseExpression - | whenExpression - | (functionName LPAREN) => function - | tableOrColumn - | LPAREN! expression RPAREN! - ; - - -precedenceFieldExpression - : - atomExpression ((LSQUARE^ expression RSQUARE!) | (DOT^ identifier))* - ; - -precedenceUnaryOperator - : - PLUS | MINUS | TILDE - ; - -nullCondition - : - KW_NULL -> ^(TOK_ISNULL) - | KW_NOT KW_NULL -> ^(TOK_ISNOTNULL) - ; - -precedenceUnaryPrefixExpression - : - (precedenceUnaryOperator^)* precedenceFieldExpression - ; - -precedenceUnarySuffixExpression - : precedenceUnaryPrefixExpression (a=KW_IS nullCondition)? - -> {$a != null}? ^(TOK_FUNCTION nullCondition precedenceUnaryPrefixExpression) - -> precedenceUnaryPrefixExpression - ; - - -precedenceBitwiseXorOperator - : - BITWISEXOR - ; - -precedenceBitwiseXorExpression - : - precedenceUnarySuffixExpression (precedenceBitwiseXorOperator^ precedenceUnarySuffixExpression)* - ; - - -precedenceStarOperator - : - STAR | DIVIDE | MOD | DIV - ; - -precedenceStarExpression - : - precedenceBitwiseXorExpression (precedenceStarOperator^ precedenceBitwiseXorExpression)* - ; - - -precedencePlusOperator - : - PLUS | MINUS - ; - -precedencePlusExpression - : - precedenceStarExpression (precedencePlusOperator^ precedenceStarExpression)* - ; - - -precedenceAmpersandOperator - : - AMPERSAND - ; - -precedenceAmpersandExpression - : - precedencePlusExpression (precedenceAmpersandOperator^ precedencePlusExpression)* - ; - - -precedenceBitwiseOrOperator - : - BITWISEOR - ; - -precedenceBitwiseOrExpression - : - precedenceAmpersandExpression (precedenceBitwiseOrOperator^ precedenceAmpersandExpression)* - ; - - -// Equal operators supporting NOT prefix -precedenceEqualNegatableOperator - : - KW_LIKE | KW_RLIKE | KW_REGEXP - ; - -precedenceEqualOperator - : - precedenceEqualNegatableOperator | EQUAL | EQUAL_NS | NOTEQUAL | LESSTHANOREQUALTO | LESSTHAN | GREATERTHANOREQUALTO | GREATERTHAN - ; - -subQueryExpression - : - LPAREN! selectStatement[true] RPAREN! - ; - -precedenceEqualExpression - : - (LPAREN precedenceBitwiseOrExpression COMMA) => precedenceEqualExpressionMutiple - | - precedenceEqualExpressionSingle - ; - -precedenceEqualExpressionSingle - : - (left=precedenceBitwiseOrExpression -> $left) - ( - (KW_NOT precedenceEqualNegatableOperator notExpr=precedenceBitwiseOrExpression) - -> ^(KW_NOT ^(precedenceEqualNegatableOperator $precedenceEqualExpressionSingle $notExpr)) - | (precedenceEqualOperator equalExpr=precedenceBitwiseOrExpression) - -> ^(precedenceEqualOperator $precedenceEqualExpressionSingle $equalExpr) - | (KW_NOT KW_IN LPAREN KW_SELECT)=> (KW_NOT KW_IN subQueryExpression) - -> ^(KW_NOT ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_IN) subQueryExpression $precedenceEqualExpressionSingle)) - | (KW_NOT KW_IN expressions) - -> ^(KW_NOT ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionSingle expressions)) - | (KW_IN LPAREN KW_SELECT)=> (KW_IN subQueryExpression) - -> ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_IN) subQueryExpression $precedenceEqualExpressionSingle) - | (KW_IN expressions) - -> ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionSingle expressions) - | ( KW_NOT KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) ) - -> ^(TOK_FUNCTION Identifier["between"] KW_TRUE $left $min $max) - | ( KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) ) - -> ^(TOK_FUNCTION Identifier["between"] KW_FALSE $left $min $max) - )* - | (KW_EXISTS LPAREN KW_SELECT)=> (KW_EXISTS subQueryExpression) -> ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_EXISTS) subQueryExpression) - ; - -expressions - : - LPAREN expression (COMMA expression)* RPAREN -> expression+ - ; - -//we transform the (col0, col1) in ((v00,v01),(v10,v11)) into struct(col0, col1) in (struct(v00,v01),struct(v10,v11)) -precedenceEqualExpressionMutiple - : - (LPAREN precedenceBitwiseOrExpression (COMMA precedenceBitwiseOrExpression)+ RPAREN -> ^(TOK_FUNCTION Identifier["struct"] precedenceBitwiseOrExpression+)) - ( (KW_IN LPAREN expressionsToStruct (COMMA expressionsToStruct)+ RPAREN) - -> ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionMutiple expressionsToStruct+) - | (KW_NOT KW_IN LPAREN expressionsToStruct (COMMA expressionsToStruct)+ RPAREN) - -> ^(KW_NOT ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionMutiple expressionsToStruct+))) - ; - -expressionsToStruct - : - LPAREN expression (COMMA expression)* RPAREN -> ^(TOK_FUNCTION Identifier["struct"] expression+) - ; - -precedenceNotOperator - : - KW_NOT - ; - -precedenceNotExpression - : - (precedenceNotOperator^)* precedenceEqualExpression - ; - - -precedenceAndOperator - : - KW_AND - ; - -precedenceAndExpression - : - precedenceNotExpression (precedenceAndOperator^ precedenceNotExpression)* - ; - - -precedenceOrOperator - : - KW_OR - ; - -precedenceOrExpression - : - precedenceAndExpression (precedenceOrOperator^ precedenceAndExpression)* - ; - - -booleanValue - : - KW_TRUE^ | KW_FALSE^ - ; - -booleanValueTok - : - KW_TRUE -> TOK_TRUE - | KW_FALSE -> TOK_FALSE - ; - -tableOrPartition - : - tableName partitionSpec? -> ^(TOK_TAB tableName partitionSpec?) - ; - -partitionSpec - : - KW_PARTITION - LPAREN partitionVal (COMMA partitionVal )* RPAREN -> ^(TOK_PARTSPEC partitionVal +) - ; - -partitionVal - : - identifier (EQUAL constant)? -> ^(TOK_PARTVAL identifier constant?) - ; - -dropPartitionSpec - : - KW_PARTITION - LPAREN dropPartitionVal (COMMA dropPartitionVal )* RPAREN -> ^(TOK_PARTSPEC dropPartitionVal +) - ; - -dropPartitionVal - : - identifier dropPartitionOperator constant -> ^(TOK_PARTVAL identifier dropPartitionOperator constant) - ; - -dropPartitionOperator - : - EQUAL | NOTEQUAL | LESSTHANOREQUALTO | LESSTHAN | GREATERTHANOREQUALTO | GREATERTHAN - ; - -sysFuncNames - : - KW_AND - | KW_OR - | KW_NOT - | KW_LIKE - | KW_IF - | KW_CASE - | KW_WHEN - | KW_TINYINT - | KW_SMALLINT - | KW_INT - | KW_BIGINT - | KW_FLOAT - | KW_DOUBLE - | KW_BOOLEAN - | KW_STRING - | KW_BINARY - | KW_ARRAY - | KW_MAP - | KW_STRUCT - | KW_UNIONTYPE - | EQUAL - | EQUAL_NS - | NOTEQUAL - | LESSTHANOREQUALTO - | LESSTHAN - | GREATERTHANOREQUALTO - | GREATERTHAN - | DIVIDE - | PLUS - | MINUS - | STAR - | MOD - | DIV - | AMPERSAND - | TILDE - | BITWISEOR - | BITWISEXOR - | KW_RLIKE - | KW_REGEXP - | KW_IN - | KW_BETWEEN - ; - -descFuncNames - : - (sysFuncNames) => sysFuncNames - | StringLiteral - | functionIdentifier - ; - -identifier - : - Identifier - | nonReserved -> Identifier[$nonReserved.text] - // If it decides to support SQL11 reserved keywords, i.e., useSQL11ReservedKeywordsForIdentifier()=false, - // the sql11keywords in existing q tests will NOT be added back. - | {useSQL11ReservedKeywordsForIdentifier()}? sql11ReservedKeywordsUsedAsIdentifier -> Identifier[$sql11ReservedKeywordsUsedAsIdentifier.text] - ; - -functionIdentifier -@init { gParent.pushMsg("function identifier", state); } -@after { gParent.popMsg(state); } - : db=identifier DOT fn=identifier - -> Identifier[$db.text + "." + $fn.text] - | - identifier - ; - -principalIdentifier -@init { gParent.pushMsg("identifier for principal spec", state); } -@after { gParent.popMsg(state); } - : identifier - | QuotedIdentifier - ; - -//The new version of nonReserved + sql11ReservedKeywordsUsedAsIdentifier = old version of nonReserved -//Non reserved keywords are basically the keywords that can be used as identifiers. -//All the KW_* are automatically not only keywords, but also reserved keywords. -//That means, they can NOT be used as identifiers. -//If you would like to use them as identifiers, put them in the nonReserved list below. -//If you are not sure, please refer to the SQL2011 column in -//http://www.postgresql.org/docs/9.5/static/sql-keywords-appendix.html -nonReserved - : - KW_ADD | KW_ADMIN | KW_AFTER | KW_ANALYZE | KW_ARCHIVE | KW_ASC | KW_BEFORE | KW_BUCKET | KW_BUCKETS - | KW_CASCADE | KW_CHANGE | KW_CLUSTER | KW_CLUSTERED | KW_CLUSTERSTATUS | KW_COLLECTION | KW_COLUMNS - | KW_COMMENT | KW_COMPACT | KW_COMPACTIONS | KW_COMPUTE | KW_CONCATENATE | KW_CONTINUE | KW_DATA | KW_DAY - | KW_DATABASES | KW_DATETIME | KW_DBPROPERTIES | KW_DEFERRED | KW_DEFINED | KW_DELIMITED | KW_DEPENDENCY - | KW_DESC | KW_DIRECTORIES | KW_DIRECTORY | KW_DISABLE | KW_DISTRIBUTE | KW_ELEM_TYPE - | KW_ENABLE | KW_ESCAPED | KW_EXCLUSIVE | KW_EXPLAIN | KW_EXPORT | KW_FIELDS | KW_FILE | KW_FILEFORMAT - | KW_FIRST | KW_FORMAT | KW_FORMATTED | KW_FUNCTIONS | KW_HOLD_DDLTIME | KW_HOUR | KW_IDXPROPERTIES | KW_IGNORE - | KW_INDEX | KW_INDEXES | KW_INPATH | KW_INPUTDRIVER | KW_INPUTFORMAT | KW_ITEMS | KW_JAR - | KW_KEYS | KW_KEY_TYPE | KW_LIMIT | KW_LINES | KW_LOAD | KW_LOCATION | KW_LOCK | KW_LOCKS | KW_LOGICAL | KW_LONG - | KW_MAPJOIN | KW_MATERIALIZED | KW_METADATA | KW_MINUS | KW_MINUTE | KW_MONTH | KW_MSCK | KW_NOSCAN | KW_NO_DROP | KW_OFFLINE - | KW_OPTION | KW_OUTPUTDRIVER | KW_OUTPUTFORMAT | KW_OVERWRITE | KW_OWNER | KW_PARTITIONED | KW_PARTITIONS | KW_PLUS | KW_PRETTY - | KW_PRINCIPALS | KW_PROTECTION | KW_PURGE | KW_READ | KW_READONLY | KW_REBUILD | KW_RECORDREADER | KW_RECORDWRITER - | KW_RELOAD | KW_RENAME | KW_REPAIR | KW_REPLACE | KW_REPLICATION | KW_RESTRICT | KW_REWRITE - | KW_ROLE | KW_ROLES | KW_SCHEMA | KW_SCHEMAS | KW_SECOND | KW_SEMI | KW_SERDE | KW_SERDEPROPERTIES | KW_SERVER | KW_SETS | KW_SHARED - | KW_SHOW | KW_SHOW_DATABASE | KW_SKEWED | KW_SORT | KW_SORTED | KW_SSL | KW_STATISTICS | KW_STORED - | KW_STREAMTABLE | KW_STRING | KW_STRUCT | KW_TABLES | KW_TBLPROPERTIES | KW_TEMPORARY | KW_TERMINATED - | KW_TINYINT | KW_TOUCH | KW_TRANSACTIONS | KW_UNARCHIVE | KW_UNDO | KW_UNIONTYPE | KW_UNLOCK | KW_UNSET - | KW_UNSIGNED | KW_URI | KW_USE | KW_UTC | KW_UTCTIMESTAMP | KW_VALUE_TYPE | KW_VIEW | KW_WHILE | KW_YEAR - | KW_WORK - | KW_TRANSACTION - | KW_WRITE - | KW_ISOLATION - | KW_LEVEL - | KW_SNAPSHOT - | KW_AUTOCOMMIT - | KW_ANTI -; - -//The following SQL2011 reserved keywords are used as cast function name only, but not as identifiers. -sql11ReservedKeywordsUsedAsCastFunctionName - : - KW_BIGINT | KW_BINARY | KW_BOOLEAN | KW_CURRENT_DATE | KW_CURRENT_TIMESTAMP | KW_DATE | KW_DOUBLE | KW_FLOAT | KW_INT | KW_SMALLINT | KW_TIMESTAMP - ; - -//The following SQL2011 reserved keywords are used as identifiers in many q tests, they may be added back due to backward compatibility. -//We are planning to remove the following whole list after several releases. -//Thus, please do not change the following list unless you know what to do. -sql11ReservedKeywordsUsedAsIdentifier - : - KW_ALL | KW_ALTER | KW_ARRAY | KW_AS | KW_AUTHORIZATION | KW_BETWEEN | KW_BIGINT | KW_BINARY | KW_BOOLEAN - | KW_BOTH | KW_BY | KW_CREATE | KW_CUBE | KW_CURRENT_DATE | KW_CURRENT_TIMESTAMP | KW_CURSOR | KW_DATE | KW_DECIMAL | KW_DELETE | KW_DESCRIBE - | KW_DOUBLE | KW_DROP | KW_EXISTS | KW_EXTERNAL | KW_FALSE | KW_FETCH | KW_FLOAT | KW_FOR | KW_FULL | KW_GRANT - | KW_GROUP | KW_GROUPING | KW_IMPORT | KW_IN | KW_INNER | KW_INSERT | KW_INT | KW_INTERSECT | KW_INTO | KW_IS | KW_LATERAL - | KW_LEFT | KW_LIKE | KW_LOCAL | KW_NONE | KW_NULL | KW_OF | KW_ORDER | KW_OUT | KW_OUTER | KW_PARTITION - | KW_PERCENT | KW_PROCEDURE | KW_RANGE | KW_READS | KW_REVOKE | KW_RIGHT - | KW_ROLLUP | KW_ROW | KW_ROWS | KW_SET | KW_SMALLINT | KW_TABLE | KW_TIMESTAMP | KW_TO | KW_TRIGGER | KW_TRUE - | KW_TRUNCATE | KW_UNION | KW_UPDATE | KW_USER | KW_USING | KW_VALUES | KW_WITH -//The following two keywords come from MySQL. Although they are not keywords in SQL2011, they are reserved keywords in MySQL. - | KW_REGEXP | KW_RLIKE - ; diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g index 98b46794a630c..4afce3090f739 100644 --- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g @@ -26,7 +26,7 @@ ASTLabelType=CommonTree; backtrack=false; k=3; } -import SelectClauseParser, FromClauseParser, IdentifiersParser; +import SelectClauseParser, FromClauseParser, IdentifiersParser, ExpressionParser; tokens { TOK_INSERT; From 917d3fc069fb9ea1c1487119c9c12b373f4f9b77 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 6 Jan 2016 16:58:10 -0800 Subject: [PATCH 012/110] [SPARK-12539][SQL] support writing bucketed table This PR adds bucket write support to Spark SQL. User can specify bucketing columns, numBuckets and sorting columns with or without partition columns. For example: ``` df.write.partitionBy("year").bucketBy(8, "country").sortBy("amount").saveAsTable("sales") ``` When bucketing is used, we will calculate bucket id for each record, and group the records by bucket id. For each group, we will create a file with bucket id in its name, and write data into it. For each bucket file, if sorting columns are specified, the data will be sorted before write. Note that there may be multiply files for one bucket, as the data is distributed. Currently we store the bucket metadata at hive metastore in a non-hive-compatible way. We use different bucketing hash function compared to hive, so we can't be compatible anyway. Limitations: * Can't write bucketed data without hive metastore. * Can't insert bucketed data into existing hive tables. Author: Wenchen Fan Closes #10498 from cloud-fan/bucket-write. --- .../apache/spark/sql/DataFrameWriter.scala | 89 ++++++- .../spark/sql/execution/SparkStrategies.scala | 7 +- .../sql/execution/datasources/DDLParser.scala | 1 + .../InsertIntoHadoopFsRelation.scala | 2 +- .../datasources/ResolvedDataSource.scala | 2 + .../datasources/WriterContainer.scala | 219 +++++++++++++----- .../sql/execution/datasources/bucket.scala | 57 +++++ .../spark/sql/execution/datasources/ddl.scala | 10 +- .../datasources/json/JSONRelation.scala | 35 ++- .../datasources/parquet/ParquetRelation.scala | 28 ++- .../sql/execution/datasources/rules.scala | 24 +- .../apache/spark/sql/sources/interfaces.scala | 34 ++- .../spark/sql/hive/HiveMetastoreCatalog.scala | 23 +- .../spark/sql/hive/HiveStrategies.scala | 7 +- .../spark/sql/hive/execution/commands.scala | 15 +- .../spark/sql/hive/orc/OrcRelation.scala | 20 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 1 + .../sql/sources/BucketedWriteSuite.scala | 169 ++++++++++++++ 18 files changed, 626 insertions(+), 117 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index e2d72a549e6b0..00f9817b53976 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -23,9 +23,9 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.{SqlParser, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation} +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, Project} -import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, ResolvedDataSource} +import org.apache.spark.sql.execution.datasources.{BucketSpec, CreateTableUsingAsSelect, ResolvedDataSource} import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils import org.apache.spark.sql.sources.HadoopFsRelation @@ -128,6 +128,34 @@ final class DataFrameWriter private[sql](df: DataFrame) { this } + /** + * Buckets the output by the given columns. If specified, the output is laid out on the file + * system similar to Hive's bucketing scheme. + * + * This is applicable for Parquet, JSON and ORC. + * + * @since 2.0 + */ + @scala.annotation.varargs + def bucketBy(numBuckets: Int, colName: String, colNames: String*): DataFrameWriter = { + this.numBuckets = Option(numBuckets) + this.bucketColumnNames = Option(colName +: colNames) + this + } + + /** + * Sorts the output in each bucket by the given columns. + * + * This is applicable for Parquet, JSON and ORC. + * + * @since 2.0 + */ + @scala.annotation.varargs + def sortBy(colName: String, colNames: String*): DataFrameWriter = { + this.sortColumnNames = Option(colName +: colNames) + this + } + /** * Saves the content of the [[DataFrame]] at the specified path. * @@ -144,10 +172,12 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 1.4.0 */ def save(): Unit = { + assertNotBucketed() ResolvedDataSource( df.sqlContext, source, partitioningColumns.map(_.toArray).getOrElse(Array.empty[String]), + getBucketSpec, mode, extraOptions.toMap, df) @@ -166,6 +196,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { } private def insertInto(tableIdent: TableIdentifier): Unit = { + assertNotBucketed() val partitions = normalizedParCols.map(_.map(col => col -> (None: Option[String])).toMap) val overwrite = mode == SaveMode.Overwrite @@ -188,13 +219,47 @@ final class DataFrameWriter private[sql](df: DataFrame) { ifNotExists = false)).toRdd } - private def normalizedParCols: Option[Seq[String]] = partitioningColumns.map { parCols => - parCols.map { col => - df.logicalPlan.output - .map(_.name) - .find(df.sqlContext.analyzer.resolver(_, col)) - .getOrElse(throw new AnalysisException(s"Partition column $col not found in existing " + - s"columns (${df.logicalPlan.output.map(_.name).mkString(", ")})")) + private def normalizedParCols: Option[Seq[String]] = partitioningColumns.map { cols => + cols.map(normalize(_, "Partition")) + } + + private def normalizedBucketColNames: Option[Seq[String]] = bucketColumnNames.map { cols => + cols.map(normalize(_, "Bucketing")) + } + + private def normalizedSortColNames: Option[Seq[String]] = sortColumnNames.map { cols => + cols.map(normalize(_, "Sorting")) + } + + private def getBucketSpec: Option[BucketSpec] = { + if (sortColumnNames.isDefined) { + require(numBuckets.isDefined, "sortBy must be used together with bucketBy") + } + + for { + n <- numBuckets + } yield { + require(n > 0 && n < 100000, "Bucket number must be greater than 0 and less than 100000.") + BucketSpec(n, normalizedBucketColNames.get, normalizedSortColNames.getOrElse(Nil)) + } + } + + /** + * The given column name may not be equal to any of the existing column names if we were in + * case-insensitive context. Normalize the given column name to the real one so that we don't + * need to care about case sensitivity afterwards. + */ + private def normalize(columnName: String, columnType: String): String = { + val validColumnNames = df.logicalPlan.output.map(_.name) + validColumnNames.find(df.sqlContext.analyzer.resolver(_, columnName)) + .getOrElse(throw new AnalysisException(s"$columnType column $columnName not found in " + + s"existing columns (${validColumnNames.mkString(", ")})")) + } + + private def assertNotBucketed(): Unit = { + if (numBuckets.isDefined || sortColumnNames.isDefined) { + throw new IllegalArgumentException( + "Currently we don't support writing bucketed data to this data source.") } } @@ -244,6 +309,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { source, temporary = false, partitioningColumns.map(_.toArray).getOrElse(Array.empty[String]), + getBucketSpec, mode, extraOptions.toMap, df.logicalPlan) @@ -372,4 +438,9 @@ final class DataFrameWriter private[sql](df: DataFrame) { private var partitioningColumns: Option[Seq[String]] = None + private var bucketColumnNames: Option[Seq[String]] = None + + private var numBuckets: Option[Int] = None + + private var sortColumnNames: Option[Seq[String]] = None } 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 6cf75bc17039c..482130a18d939 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 @@ -382,13 +382,12 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case c: CreateTableUsing if c.temporary && c.allowExisting => sys.error("allowExisting should be set to false when creating a temporary table.") - case CreateTableUsingAsSelect(tableIdent, provider, true, partitionsCols, mode, opts, query) - if partitionsCols.nonEmpty => + case c: CreateTableUsingAsSelect if c.temporary && c.partitionColumns.nonEmpty => sys.error("Cannot create temporary partitioned table.") - case CreateTableUsingAsSelect(tableIdent, provider, true, _, mode, opts, query) => + case c: CreateTableUsingAsSelect if c.temporary => val cmd = CreateTempTableUsingAsSelect( - tableIdent, provider, Array.empty[String], mode, opts, query) + c.tableIdent, c.provider, Array.empty[String], c.mode, c.options, c.child) ExecutedCommand(cmd) :: Nil case c: CreateTableUsingAsSelect if !c.temporary => sys.error("Tables created with SQLContext must be TEMPORARY. Use a HiveContext instead.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala index 48eff62b297f2..d8d21b06b8b35 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala @@ -109,6 +109,7 @@ class DDLParser(parseQuery: String => LogicalPlan) provider, temp.isDefined, Array.empty[String], + bucketSpec = None, mode, options, queryPlan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala index 38152d0cf1a48..7a8691e7cb9c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala @@ -125,7 +125,7 @@ private[sql] case class InsertIntoHadoopFsRelation( |Actual: ${partitionColumns.mkString(", ")} """.stripMargin) - val writerContainer = if (partitionColumns.isEmpty) { + val writerContainer = if (partitionColumns.isEmpty && relation.bucketSpec.isEmpty) { new DefaultWriterContainer(relation, job, isAppend) } else { val output = df.queryExecution.executedPlan.output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala index 0ca0a38f712ce..ece9b8a9a9174 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala @@ -210,6 +210,7 @@ object ResolvedDataSource extends Logging { sqlContext: SQLContext, provider: String, partitionColumns: Array[String], + bucketSpec: Option[BucketSpec], mode: SaveMode, options: Map[String, String], data: DataFrame): ResolvedDataSource = { @@ -244,6 +245,7 @@ object ResolvedDataSource extends Logging { Array(outputPath.toString), Some(dataSchema.asNullable), Some(partitionColumnsSchema(data.schema, partitionColumns, caseSensitive)), + bucketSpec, caseInsensitiveOptions) // For partitioned relation r, r.schema's column ordering can be different from the column diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index 9f23d531072aa..4f8524f4b967c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.UnsafeKVExternalSorter import org.apache.spark.sql.sources.{HadoopFsRelation, OutputWriter, OutputWriterFactory} -import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.sql.types.{IntegerType, StructType, StringType} import org.apache.spark.util.SerializableConfiguration @@ -121,9 +121,9 @@ private[sql] abstract class BaseWriterContainer( } } - protected def newOutputWriter(path: String): OutputWriter = { + protected def newOutputWriter(path: String, bucketId: Option[Int] = None): OutputWriter = { try { - outputWriterFactory.newInstance(path, dataSchema, taskAttemptContext) + outputWriterFactory.newInstance(path, bucketId, dataSchema, taskAttemptContext) } catch { case e: org.apache.hadoop.fs.FileAlreadyExistsException => if (outputCommitter.isInstanceOf[parquet.DirectParquetOutputCommitter]) { @@ -312,19 +312,23 @@ private[sql] class DynamicPartitionWriterContainer( isAppend: Boolean) extends BaseWriterContainer(relation, job, isAppend) { - def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { - val outputWriters = new java.util.HashMap[InternalRow, OutputWriter] - executorSideSetup(taskContext) + private val bucketSpec = relation.bucketSpec - var outputWritersCleared = false + private val bucketColumns: Seq[Attribute] = bucketSpec.toSeq.flatMap { + spec => spec.bucketColumnNames.map(c => inputSchema.find(_.name == c).get) + } - // Returns the partition key given an input row - val getPartitionKey = UnsafeProjection.create(partitionColumns, inputSchema) - // Returns the data columns to be written given an input row - val getOutputRow = UnsafeProjection.create(dataColumns, inputSchema) + private val sortColumns: Seq[Attribute] = bucketSpec.toSeq.flatMap { + spec => spec.sortColumnNames.map(c => inputSchema.find(_.name == c).get) + } + + private def bucketIdExpression: Option[Expression] = for { + BucketSpec(numBuckets, _, _) <- bucketSpec + } yield Pmod(new Murmur3Hash(bucketColumns), Literal(numBuckets)) - // Expressions that given a partition key build a string like: col1=val/col2=val/... - val partitionStringExpression = partitionColumns.zipWithIndex.flatMap { case (c, i) => + // Expressions that given a partition key build a string like: col1=val/col2=val/... + private def partitionStringExpression: Seq[Expression] = { + partitionColumns.zipWithIndex.flatMap { case (c, i) => val escaped = ScalaUDF( PartitioningUtils.escapePathName _, @@ -335,6 +339,121 @@ private[sql] class DynamicPartitionWriterContainer( val partitionName = Literal(c.name + "=") :: str :: Nil if (i == 0) partitionName else Literal(Path.SEPARATOR) :: partitionName } + } + + private def getBucketIdFromKey(key: InternalRow): Option[Int] = { + if (bucketSpec.isDefined) { + Some(key.getInt(partitionColumns.length)) + } else { + None + } + } + + private def sameBucket(key1: UnsafeRow, key2: UnsafeRow): Boolean = { + val bucketIdIndex = partitionColumns.length + if (key1.getInt(bucketIdIndex) != key2.getInt(bucketIdIndex)) { + false + } else { + var i = partitionColumns.length - 1 + while (i >= 0) { + val dt = partitionColumns(i).dataType + if (key1.get(i, dt) != key2.get(i, dt)) return false + i -= 1 + } + true + } + } + + private def sortBasedWrite( + sorter: UnsafeKVExternalSorter, + iterator: Iterator[InternalRow], + getSortingKey: UnsafeProjection, + getOutputRow: UnsafeProjection, + getPartitionString: UnsafeProjection, + outputWriters: java.util.HashMap[InternalRow, OutputWriter]): Unit = { + while (iterator.hasNext) { + val currentRow = iterator.next() + sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) + } + + logInfo(s"Sorting complete. Writing out partition files one at a time.") + + val needNewWriter: (UnsafeRow, UnsafeRow) => Boolean = if (sortColumns.isEmpty) { + (key1, key2) => key1 != key2 + } else { + (key1, key2) => key1 == null || !sameBucket(key1, key2) + } + + val sortedIterator = sorter.sortedIterator() + var currentKey: UnsafeRow = null + var currentWriter: OutputWriter = null + try { + while (sortedIterator.next()) { + if (needNewWriter(currentKey, sortedIterator.getKey)) { + if (currentWriter != null) { + currentWriter.close() + } + currentKey = sortedIterator.getKey.copy() + logDebug(s"Writing partition: $currentKey") + + // Either use an existing file from before, or open a new one. + currentWriter = outputWriters.remove(currentKey) + if (currentWriter == null) { + currentWriter = newOutputWriter(currentKey, getPartitionString) + } + } + + currentWriter.writeInternal(sortedIterator.getValue) + } + } finally { + if (currentWriter != null) { currentWriter.close() } + } + } + + /** + * Open and returns a new OutputWriter given a partition key and optional bucket id. + * If bucket id is specified, we will append it to the end of the file name, but before the + * file extension, e.g. part-r-00009-ea518ad4-455a-4431-b471-d24e03814677-00002.gz.parquet + */ + private def newOutputWriter( + key: InternalRow, + getPartitionString: UnsafeProjection): OutputWriter = { + val configuration = taskAttemptContext.getConfiguration + val path = if (partitionColumns.nonEmpty) { + val partitionPath = getPartitionString(key).getString(0) + configuration.set( + "spark.sql.sources.output.path", new Path(outputPath, partitionPath).toString) + new Path(getWorkPath, partitionPath).toString + } else { + configuration.set("spark.sql.sources.output.path", outputPath) + getWorkPath + } + val bucketId = getBucketIdFromKey(key) + val newWriter = super.newOutputWriter(path, bucketId) + newWriter.initConverter(dataSchema) + newWriter + } + + def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { + val outputWriters = new java.util.HashMap[InternalRow, OutputWriter] + executorSideSetup(taskContext) + + var outputWritersCleared = false + + // We should first sort by partition columns, then bucket id, and finally sorting columns. + val getSortingKey = + UnsafeProjection.create(partitionColumns ++ bucketIdExpression ++ sortColumns, inputSchema) + + val sortingKeySchema = if (bucketSpec.isEmpty) { + StructType.fromAttributes(partitionColumns) + } else { // If it's bucketed, we should also consider bucket id as part of the key. + val fields = StructType.fromAttributes(partitionColumns) + .add("bucketId", IntegerType, nullable = false) ++ StructType.fromAttributes(sortColumns) + StructType(fields) + } + + // Returns the data columns to be written given an input row + val getOutputRow = UnsafeProjection.create(dataColumns, inputSchema) // Returns the partition path given a partition key. val getPartitionString = @@ -342,22 +461,34 @@ private[sql] class DynamicPartitionWriterContainer( // If anything below fails, we should abort the task. try { - // This will be filled in if we have to fall back on sorting. - var sorter: UnsafeKVExternalSorter = null + // If there is no sorting columns, we set sorter to null and try the hash-based writing first, + // and fill the sorter if there are too many writers and we need to fall back on sorting. + // If there are sorting columns, then we have to sort the data anyway, and no need to try the + // hash-based writing first. + var sorter: UnsafeKVExternalSorter = if (sortColumns.nonEmpty) { + new UnsafeKVExternalSorter( + sortingKeySchema, + StructType.fromAttributes(dataColumns), + SparkEnv.get.blockManager, + TaskContext.get().taskMemoryManager().pageSizeBytes) + } else { + null + } while (iterator.hasNext && sorter == null) { val inputRow = iterator.next() - val currentKey = getPartitionKey(inputRow) + // When we reach here, the `sortColumns` must be empty, so the sorting key is hashing key. + val currentKey = getSortingKey(inputRow) var currentWriter = outputWriters.get(currentKey) if (currentWriter == null) { if (outputWriters.size < maxOpenFiles) { - currentWriter = newOutputWriter(currentKey) + currentWriter = newOutputWriter(currentKey, getPartitionString) outputWriters.put(currentKey.copy(), currentWriter) currentWriter.writeInternal(getOutputRow(inputRow)) } else { logInfo(s"Maximum partitions reached, falling back on sorting.") sorter = new UnsafeKVExternalSorter( - StructType.fromAttributes(partitionColumns), + sortingKeySchema, StructType.fromAttributes(dataColumns), SparkEnv.get.blockManager, TaskContext.get().taskMemoryManager().pageSizeBytes) @@ -369,39 +500,15 @@ private[sql] class DynamicPartitionWriterContainer( } // If the sorter is not null that means that we reached the maxFiles above and need to finish - // using external sort. + // using external sort, or there are sorting columns and we need to sort the whole data set. if (sorter != null) { - while (iterator.hasNext) { - val currentRow = iterator.next() - sorter.insertKV(getPartitionKey(currentRow), getOutputRow(currentRow)) - } - - logInfo(s"Sorting complete. Writing out partition files one at a time.") - - val sortedIterator = sorter.sortedIterator() - var currentKey: InternalRow = null - var currentWriter: OutputWriter = null - try { - while (sortedIterator.next()) { - if (currentKey != sortedIterator.getKey) { - if (currentWriter != null) { - currentWriter.close() - } - currentKey = sortedIterator.getKey.copy() - logDebug(s"Writing partition: $currentKey") - - // Either use an existing file from before, or open a new one. - currentWriter = outputWriters.remove(currentKey) - if (currentWriter == null) { - currentWriter = newOutputWriter(currentKey) - } - } - - currentWriter.writeInternal(sortedIterator.getValue) - } - } finally { - if (currentWriter != null) { currentWriter.close() } - } + sortBasedWrite( + sorter, + iterator, + getSortingKey, + getOutputRow, + getPartitionString, + outputWriters) } commitTask() @@ -412,18 +519,6 @@ private[sql] class DynamicPartitionWriterContainer( throw new SparkException("Task failed while writing rows.", cause) } - /** Open and returns a new OutputWriter given a partition key. */ - def newOutputWriter(key: InternalRow): OutputWriter = { - val partitionPath = getPartitionString(key).getString(0) - val path = new Path(getWorkPath, partitionPath) - val configuration = taskAttemptContext.getConfiguration - configuration.set( - "spark.sql.sources.output.path", new Path(outputPath, partitionPath).toString) - val newWriter = super.newOutputWriter(path.toString) - newWriter.initConverter(dataSchema) - newWriter - } - def clearOutputWriters(): Unit = { if (!outputWritersCleared) { outputWriters.asScala.values.foreach(_.close()) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala new file mode 100644 index 0000000000000..82287c8967134 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.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.execution.datasources + +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.sources.{OutputWriter, OutputWriterFactory, HadoopFsRelationProvider, HadoopFsRelation} +import org.apache.spark.sql.types.StructType + +/** + * A container for bucketing information. + * Bucketing is a technology for decomposing data sets into more manageable parts, and the number + * of buckets is fixed so it does not fluctuate with data. + * + * @param numBuckets number of buckets. + * @param bucketColumnNames the names of the columns that used to generate the bucket id. + * @param sortColumnNames the names of the columns that used to sort data in each bucket. + */ +private[sql] case class BucketSpec( + numBuckets: Int, + bucketColumnNames: Seq[String], + sortColumnNames: Seq[String]) + +private[sql] trait BucketedHadoopFsRelationProvider extends HadoopFsRelationProvider { + final override def createRelation( + sqlContext: SQLContext, + paths: Array[String], + dataSchema: Option[StructType], + partitionColumns: Option[StructType], + parameters: Map[String, String]): HadoopFsRelation = + // TODO: throw exception here as we won't call this method during execution, after bucketed read + // support is finished. + createRelation(sqlContext, paths, dataSchema, partitionColumns, bucketSpec = None, parameters) +} + +private[sql] abstract class BucketedOutputWriterFactory extends OutputWriterFactory { + final override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = + throw new UnsupportedOperationException("use bucket version") +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index aed5d0dcf2d8a..0897fcadbc011 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -76,6 +76,7 @@ case class CreateTableUsingAsSelect( provider: String, temporary: Boolean, partitionColumns: Array[String], + bucketSpec: Option[BucketSpec], mode: SaveMode, options: Map[String, String], child: LogicalPlan) extends UnaryNode { @@ -109,7 +110,14 @@ case class CreateTempTableUsingAsSelect( override def run(sqlContext: SQLContext): Seq[Row] = { val df = DataFrame(sqlContext, query) - val resolved = ResolvedDataSource(sqlContext, provider, partitionColumns, mode, options, df) + val resolved = ResolvedDataSource( + sqlContext, + provider, + partitionColumns, + bucketSpec = None, + mode, + options, + df) sqlContext.catalog.registerTable( tableIdent, DataFrame(sqlContext, LogicalRelation(resolved.relation)).logicalPlan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala index 8bf538178b5d9..b92edf65bfb6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala @@ -34,13 +34,13 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Row, SQLContext} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeProjection -import org.apache.spark.sql.execution.datasources.PartitionSpec +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration -class DefaultSource extends HadoopFsRelationProvider with DataSourceRegister { +class DefaultSource extends BucketedHadoopFsRelationProvider with DataSourceRegister { override def shortName(): String = "json" @@ -49,6 +49,7 @@ class DefaultSource extends HadoopFsRelationProvider with DataSourceRegister { paths: Array[String], dataSchema: Option[StructType], partitionColumns: Option[StructType], + bucketSpec: Option[BucketSpec], parameters: Map[String, String]): HadoopFsRelation = { new JSONRelation( @@ -56,6 +57,7 @@ class DefaultSource extends HadoopFsRelationProvider with DataSourceRegister { maybeDataSchema = dataSchema, maybePartitionSpec = None, userDefinedPartitionColumns = partitionColumns, + bucketSpec = bucketSpec, paths = paths, parameters = parameters)(sqlContext) } @@ -66,11 +68,29 @@ private[sql] class JSONRelation( val maybeDataSchema: Option[StructType], val maybePartitionSpec: Option[PartitionSpec], override val userDefinedPartitionColumns: Option[StructType], + override val bucketSpec: Option[BucketSpec], override val paths: Array[String] = Array.empty[String], parameters: Map[String, String] = Map.empty[String, String]) (@transient val sqlContext: SQLContext) extends HadoopFsRelation(maybePartitionSpec, parameters) { + def this( + inputRDD: Option[RDD[String]], + maybeDataSchema: Option[StructType], + maybePartitionSpec: Option[PartitionSpec], + userDefinedPartitionColumns: Option[StructType], + paths: Array[String] = Array.empty[String], + parameters: Map[String, String] = Map.empty[String, String])(sqlContext: SQLContext) = { + this( + inputRDD, + maybeDataSchema, + maybePartitionSpec, + userDefinedPartitionColumns, + None, + paths, + parameters)(sqlContext) + } + val options: JSONOptions = JSONOptions.createFromConfigMap(parameters) /** Constraints to be imposed on schema to be stored. */ @@ -158,13 +178,14 @@ private[sql] class JSONRelation( partitionColumns) } - override def prepareJobForWrite(job: Job): OutputWriterFactory = { - new OutputWriterFactory { + override def prepareJobForWrite(job: Job): BucketedOutputWriterFactory = { + new BucketedOutputWriterFactory { override def newInstance( path: String, + bucketId: Option[Int], dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new JsonOutputWriter(path, dataSchema, context) + new JsonOutputWriter(path, bucketId, dataSchema, context) } } } @@ -172,6 +193,7 @@ private[sql] class JSONRelation( private[json] class JsonOutputWriter( path: String, + bucketId: Option[Int], dataSchema: StructType, context: TaskAttemptContext) extends OutputWriter with Logging { @@ -188,7 +210,8 @@ private[json] class JsonOutputWriter( val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") val taskAttemptId = context.getTaskAttemptID val split = taskAttemptId.getTaskID.getId - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$extension") + val bucketString = bucketId.map(id => f"-$id%05d").getOrElse("") + new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$bucketString$extension") } }.getRecordWriter(context) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index 45f1dff96db08..4b375de05e9e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -45,13 +45,13 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{RDD, SqlNewHadoopPartition, SqlNewHadoopRDD} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.catalyst.util.LegacyTypeStringParser -import org.apache.spark.sql.execution.datasources.PartitionSpec import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.{SerializableConfiguration, Utils} -private[sql] class DefaultSource extends HadoopFsRelationProvider with DataSourceRegister { +private[sql] class DefaultSource extends BucketedHadoopFsRelationProvider with DataSourceRegister { override def shortName(): String = "parquet" @@ -60,13 +60,17 @@ private[sql] class DefaultSource extends HadoopFsRelationProvider with DataSourc paths: Array[String], schema: Option[StructType], partitionColumns: Option[StructType], + bucketSpec: Option[BucketSpec], parameters: Map[String, String]): HadoopFsRelation = { - new ParquetRelation(paths, schema, None, partitionColumns, parameters)(sqlContext) + new ParquetRelation(paths, schema, None, partitionColumns, bucketSpec, parameters)(sqlContext) } } // NOTE: This class is instantiated and used on executor side only, no need to be serializable. -private[sql] class ParquetOutputWriter(path: String, context: TaskAttemptContext) +private[sql] class ParquetOutputWriter( + path: String, + bucketId: Option[Int], + context: TaskAttemptContext) extends OutputWriter { private val recordWriter: RecordWriter[Void, InternalRow] = { @@ -86,7 +90,8 @@ private[sql] class ParquetOutputWriter(path: String, context: TaskAttemptContext val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") val taskAttemptId = context.getTaskAttemptID val split = taskAttemptId.getTaskID.getId - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$extension") + val bucketString = bucketId.map(id => f"-$id%05d").getOrElse("") + new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$bucketString$extension") } } } @@ -107,6 +112,7 @@ private[sql] class ParquetRelation( // This is for metastore conversion. private val maybePartitionSpec: Option[PartitionSpec], override val userDefinedPartitionColumns: Option[StructType], + override val bucketSpec: Option[BucketSpec], parameters: Map[String, String])( val sqlContext: SQLContext) extends HadoopFsRelation(maybePartitionSpec, parameters) @@ -123,6 +129,7 @@ private[sql] class ParquetRelation( maybeDataSchema, maybePartitionSpec, maybePartitionSpec.map(_.partitionColumns), + None, parameters)(sqlContext) } @@ -216,7 +223,7 @@ private[sql] class ParquetRelation( override def sizeInBytes: Long = metadataCache.dataStatuses.map(_.getLen).sum - override def prepareJobForWrite(job: Job): OutputWriterFactory = { + override def prepareJobForWrite(job: Job): BucketedOutputWriterFactory = { val conf = ContextUtil.getConfiguration(job) // SPARK-9849 DirectParquetOutputCommitter qualified name should be backward compatible @@ -276,10 +283,13 @@ private[sql] class ParquetRelation( sqlContext.conf.parquetCompressionCodec.toUpperCase, CompressionCodecName.UNCOMPRESSED).name()) - new OutputWriterFactory { + new BucketedOutputWriterFactory { override def newInstance( - path: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new ParquetOutputWriter(path, context) + path: String, + bucketId: Option[Int], + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new ParquetOutputWriter(path, bucketId, context) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 50ecbd35760d8..d484403d1c641 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql.{AnalysisException, SaveMode, SQLContext} import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast} +import org.apache.spark.sql.catalyst.expressions.{RowOrdering, Alias, Attribute, Cast} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule @@ -165,22 +165,22 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => // OK } - case CreateTableUsingAsSelect(tableIdent, _, _, partitionColumns, mode, _, query) => + case c: CreateTableUsingAsSelect => // When the SaveMode is Overwrite, we need to check if the table is an input table of // the query. If so, we will throw an AnalysisException to let users know it is not allowed. - if (mode == SaveMode.Overwrite && catalog.tableExists(tableIdent)) { + if (c.mode == SaveMode.Overwrite && catalog.tableExists(c.tableIdent)) { // Need to remove SubQuery operator. - EliminateSubQueries(catalog.lookupRelation(tableIdent)) match { + EliminateSubQueries(catalog.lookupRelation(c.tableIdent)) match { // Only do the check if the table is a data source table // (the relation is a BaseRelation). case l @ LogicalRelation(dest: BaseRelation, _) => // Get all input data source relations of the query. - val srcRelations = query.collect { + val srcRelations = c.child.collect { case LogicalRelation(src: BaseRelation, _) => src } if (srcRelations.contains(dest)) { failAnalysis( - s"Cannot overwrite table $tableIdent that is also being read from.") + s"Cannot overwrite table ${c.tableIdent} that is also being read from.") } else { // OK } @@ -192,7 +192,17 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => } PartitioningUtils.validatePartitionColumnDataTypes( - query.schema, partitionColumns, catalog.conf.caseSensitiveAnalysis) + c.child.schema, c.partitionColumns, catalog.conf.caseSensitiveAnalysis) + + for { + spec <- c.bucketSpec + sortColumnName <- spec.sortColumnNames + sortColumn <- c.child.schema.find(_.name == sortColumnName) + } { + if (!RowOrdering.isOrderable(sortColumn.dataType)) { + failAnalysis(s"Cannot use ${sortColumn.dataType.simpleString} for sorting column.") + } + } case _ => // OK } 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 index f4c7f0a269323..c35f33132f602 100644 --- 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 @@ -21,7 +21,7 @@ import scala.collection.mutable import scala.util.Try import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection import org.apache.spark.sql.execution.{FileRelation, RDDConversions} -import org.apache.spark.sql.execution.datasources.{Partition, PartitioningUtils, PartitionSpec} +import org.apache.spark.sql.execution.datasources.{BucketSpec, Partition, PartitioningUtils, PartitionSpec} import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.util.SerializableConfiguration @@ -161,6 +161,20 @@ trait HadoopFsRelationProvider { dataSchema: Option[StructType], partitionColumns: Option[StructType], parameters: Map[String, String]): HadoopFsRelation + + // TODO: expose bucket API to users. + private[sql] def createRelation( + sqlContext: SQLContext, + paths: Array[String], + dataSchema: Option[StructType], + partitionColumns: Option[StructType], + bucketSpec: Option[BucketSpec], + parameters: Map[String, String]): HadoopFsRelation = { + if (bucketSpec.isDefined) { + throw new AnalysisException("Currently we don't support bucketing for this data source.") + } + createRelation(sqlContext, paths, dataSchema, partitionColumns, parameters) + } } /** @@ -351,7 +365,18 @@ abstract class OutputWriterFactory extends Serializable { * * @since 1.4.0 */ - def newInstance(path: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter + def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter + + // TODO: expose bucket API to users. + private[sql] def newInstance( + path: String, + bucketId: Option[Int], + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = + newInstance(path, dataSchema, context) } /** @@ -435,6 +460,9 @@ abstract class HadoopFsRelation private[sql]( private var _partitionSpec: PartitionSpec = _ + // TODO: expose bucket API to users. + private[sql] def bucketSpec: Option[BucketSpec] = None + private class FileStatusCache { var leafFiles = mutable.LinkedHashMap.empty[Path, FileStatus] 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 1616c4595221d..43d84d507b20e 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 @@ -40,7 +40,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.util.DataTypeParser import org.apache.spark.sql.execution.{datasources, FileRelation} -import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation, Partition => ParquetPartition, PartitionSpec, ResolvedDataSource} +import org.apache.spark.sql.execution.datasources.{Partition => ParquetPartition, _} import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.hive.execution.HiveNativeCommand @@ -211,6 +211,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive tableIdent: TableIdentifier, userSpecifiedSchema: Option[StructType], partitionColumns: Array[String], + bucketSpec: Option[BucketSpec], provider: String, options: Map[String, String], isExternal: Boolean): Unit = { @@ -240,6 +241,25 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive } } + if (userSpecifiedSchema.isDefined && bucketSpec.isDefined) { + val BucketSpec(numBuckets, bucketColumnNames, sortColumnNames) = bucketSpec.get + + tableProperties.put("spark.sql.sources.schema.numBuckets", numBuckets.toString) + tableProperties.put("spark.sql.sources.schema.numBucketCols", + bucketColumnNames.length.toString) + bucketColumnNames.zipWithIndex.foreach { case (bucketCol, index) => + tableProperties.put(s"spark.sql.sources.schema.bucketCol.$index", bucketCol) + } + + if (sortColumnNames.nonEmpty) { + tableProperties.put("spark.sql.sources.schema.numSortCols", + sortColumnNames.length.toString) + sortColumnNames.zipWithIndex.foreach { case (sortCol, index) => + tableProperties.put(s"spark.sql.sources.schema.sortCol.$index", sortCol) + } + } + } + if (userSpecifiedSchema.isEmpty && partitionColumns.length > 0) { // The table does not have a specified schema, which means that the schema will be inferred // when we load the table. So, we are not expecting partition columns and we will discover @@ -596,6 +616,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive conf.defaultDataSourceName, temporary = false, Array.empty[String], + bucketSpec = None, mode, options = Map.empty[String, String], 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 0b4f5a0fd6ea6..3687dd6f5a7ab 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 @@ -88,10 +88,9 @@ private[hive] trait HiveStrategies { tableIdent, userSpecifiedSchema, provider, opts, allowExisting, managedIfNoPath) ExecutedCommand(cmd) :: Nil - case CreateTableUsingAsSelect( - tableIdent, provider, false, partitionCols, mode, opts, query) => - val cmd = - CreateMetastoreDataSourceAsSelect(tableIdent, provider, partitionCols, mode, opts, query) + case c: CreateTableUsingAsSelect => + val cmd = CreateMetastoreDataSourceAsSelect(c.tableIdent, c.provider, c.partitionColumns, + c.bucketSpec, c.mode, c.options, c.child) ExecutedCommand(cmd) :: Nil case _ => 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 94210a5394f9b..612f01cda88ba 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 @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.RunnableCommand -import org.apache.spark.sql.execution.datasources.{LogicalRelation, ResolvedDataSource} +import org.apache.spark.sql.execution.datasources.{BucketSpec, LogicalRelation, ResolvedDataSource} import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -151,6 +151,7 @@ case class CreateMetastoreDataSource( tableIdent, userSpecifiedSchema, Array.empty[String], + bucketSpec = None, provider, optionsWithPath, isExternal) @@ -164,6 +165,7 @@ case class CreateMetastoreDataSourceAsSelect( tableIdent: TableIdentifier, provider: String, partitionColumns: Array[String], + bucketSpec: Option[BucketSpec], mode: SaveMode, options: Map[String, String], query: LogicalPlan) extends RunnableCommand { @@ -254,8 +256,14 @@ case class CreateMetastoreDataSourceAsSelect( } // Create the relation based on the data of df. - val resolved = - ResolvedDataSource(sqlContext, provider, partitionColumns, mode, optionsWithPath, df) + val resolved = ResolvedDataSource( + sqlContext, + provider, + partitionColumns, + bucketSpec, + mode, + optionsWithPath, + df) if (createMetastoreTable) { // We will use the schema of resolved.relation as the schema of the table (instead of @@ -265,6 +273,7 @@ case class CreateMetastoreDataSourceAsSelect( tableIdent, Some(resolved.relation.schema), partitionColumns, + bucketSpec, provider, optionsWithPath, isExternal) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index 3538d642d5231..14fa152c2331d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -37,13 +37,13 @@ import org.apache.spark.rdd.{HadoopRDD, RDD} import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.datasources.PartitionSpec +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hive.{HiveContext, HiveInspectors, HiveMetastoreTypes, HiveShim} import org.apache.spark.sql.sources.{Filter, _} import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration -private[sql] class DefaultSource extends HadoopFsRelationProvider with DataSourceRegister { +private[sql] class DefaultSource extends BucketedHadoopFsRelationProvider with DataSourceRegister { override def shortName(): String = "orc" @@ -52,17 +52,19 @@ private[sql] class DefaultSource extends HadoopFsRelationProvider with DataSourc paths: Array[String], dataSchema: Option[StructType], partitionColumns: Option[StructType], + bucketSpec: Option[BucketSpec], parameters: Map[String, String]): HadoopFsRelation = { assert( sqlContext.isInstanceOf[HiveContext], "The ORC data source can only be used with HiveContext.") - new OrcRelation(paths, dataSchema, None, partitionColumns, parameters)(sqlContext) + new OrcRelation(paths, dataSchema, None, partitionColumns, bucketSpec, parameters)(sqlContext) } } private[orc] class OrcOutputWriter( path: String, + bucketId: Option[Int], dataSchema: StructType, context: TaskAttemptContext) extends OutputWriter with HiveInspectors { @@ -101,7 +103,8 @@ private[orc] class OrcOutputWriter( val uniqueWriteJobId = conf.get("spark.sql.sources.writeJobUUID") val taskAttemptId = context.getTaskAttemptID val partition = taskAttemptId.getTaskID.getId - val filename = f"part-r-$partition%05d-$uniqueWriteJobId.orc" + val bucketString = bucketId.map(id => f"-$id%05d").getOrElse("") + val filename = f"part-r-$partition%05d-$uniqueWriteJobId$bucketString.orc" new OrcOutputFormat().getRecordWriter( new Path(path, filename).getFileSystem(conf), @@ -153,6 +156,7 @@ private[sql] class OrcRelation( maybeDataSchema: Option[StructType], maybePartitionSpec: Option[PartitionSpec], override val userDefinedPartitionColumns: Option[StructType], + override val bucketSpec: Option[BucketSpec], parameters: Map[String, String])( @transient val sqlContext: SQLContext) extends HadoopFsRelation(maybePartitionSpec, parameters) @@ -169,6 +173,7 @@ private[sql] class OrcRelation( maybeDataSchema, maybePartitionSpec, maybePartitionSpec.map(_.partitionColumns), + None, parameters)(sqlContext) } @@ -205,7 +210,7 @@ private[sql] class OrcRelation( OrcTableScan(output, this, filters, inputPaths).execute() } - override def prepareJobForWrite(job: Job): OutputWriterFactory = { + override def prepareJobForWrite(job: Job): BucketedOutputWriterFactory = { job.getConfiguration match { case conf: JobConf => conf.setOutputFormat(classOf[OrcOutputFormat]) @@ -216,12 +221,13 @@ private[sql] class OrcRelation( classOf[MapRedOutputFormat[_, _]]) } - new OutputWriterFactory { + new BucketedOutputWriterFactory { override def newInstance( path: String, + bucketId: Option[Int], dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new OrcOutputWriter(path, dataSchema, context) + new OrcOutputWriter(path, bucketId, dataSchema, context) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index e22dac3bc9e87..202851ae1366e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -707,6 +707,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv tableIdent = TableIdentifier("wide_schema"), userSpecifiedSchema = Some(schema), partitionColumns = Array.empty[String], + bucketSpec = None, provider = "json", options = Map("path" -> "just a dummy path"), isExternal = false) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala new file mode 100644 index 0000000000000..579da0291f291 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.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.sources + +import java.io.File + +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.{AnalysisException, QueryTest} + +class BucketedWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + import testImplicits._ + + test("bucketed by non-existing column") { + val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") + intercept[AnalysisException](df.write.bucketBy(2, "k").saveAsTable("tt")) + } + + test("numBuckets not greater than 0 or less than 100000") { + val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") + intercept[IllegalArgumentException](df.write.bucketBy(0, "i").saveAsTable("tt")) + intercept[IllegalArgumentException](df.write.bucketBy(100000, "i").saveAsTable("tt")) + } + + test("specify sorting columns without bucketing columns") { + val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") + intercept[IllegalArgumentException](df.write.sortBy("j").saveAsTable("tt")) + } + + test("sorting by non-orderable column") { + val df = Seq("a" -> Map(1 -> 1), "b" -> Map(2 -> 2)).toDF("i", "j") + intercept[AnalysisException](df.write.bucketBy(2, "i").sortBy("j").saveAsTable("tt")) + } + + test("write bucketed data to unsupported data source") { + val df = Seq(Tuple1("a"), Tuple1("b")).toDF("i") + intercept[AnalysisException](df.write.bucketBy(3, "i").format("text").saveAsTable("tt")) + } + + test("write bucketed data to non-hive-table or existing hive table") { + val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") + intercept[IllegalArgumentException](df.write.bucketBy(2, "i").parquet("/tmp/path")) + intercept[IllegalArgumentException](df.write.bucketBy(2, "i").json("/tmp/path")) + intercept[IllegalArgumentException](df.write.bucketBy(2, "i").insertInto("tt")) + } + + private val testFileName = """.*-(\d+)$""".r + private val otherFileName = """.*-(\d+)\..*""".r + private def getBucketId(fileName: String): Int = { + fileName match { + case testFileName(bucketId) => bucketId.toInt + case otherFileName(bucketId) => bucketId.toInt + } + } + + private def testBucketing( + dataDir: File, + source: String, + bucketCols: Seq[String], + sortCols: Seq[String] = Nil): Unit = { + val allBucketFiles = dataDir.listFiles().filterNot(f => + f.getName.startsWith(".") || f.getName.startsWith("_") + ) + val groupedBucketFiles = allBucketFiles.groupBy(f => getBucketId(f.getName)) + assert(groupedBucketFiles.size <= 8) + + for ((bucketId, bucketFiles) <- groupedBucketFiles) { + for (bucketFile <- bucketFiles) { + val df = sqlContext.read.format(source).load(bucketFile.getAbsolutePath) + .select((bucketCols ++ sortCols).map(col): _*) + + if (sortCols.nonEmpty) { + checkAnswer(df.sort(sortCols.map(col): _*), df.collect()) + } + + val rows = df.select(bucketCols.map(col): _*).queryExecution.toRdd.map(_.copy()).collect() + + for (row <- rows) { + assert(row.isInstanceOf[UnsafeRow]) + val actualBucketId = (row.hashCode() % 8 + 8) % 8 + assert(actualBucketId == bucketId) + } + } + } + } + + private val df = (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k") + + test("write bucketed data") { + for (source <- Seq("parquet", "json", "orc")) { + withTable("bucketed_table") { + df.write + .format(source) + .partitionBy("i") + .bucketBy(8, "j", "k") + .saveAsTable("bucketed_table") + + val tableDir = new File(hiveContext.warehousePath, "bucketed_table") + for (i <- 0 until 5) { + testBucketing(new File(tableDir, s"i=$i"), source, Seq("j", "k")) + } + } + } + } + + test("write bucketed data with sortBy") { + for (source <- Seq("parquet", "json", "orc")) { + withTable("bucketed_table") { + df.write + .format(source) + .partitionBy("i") + .bucketBy(8, "j") + .sortBy("k") + .saveAsTable("bucketed_table") + + val tableDir = new File(hiveContext.warehousePath, "bucketed_table") + for (i <- 0 until 5) { + testBucketing(new File(tableDir, s"i=$i"), source, Seq("j"), Seq("k")) + } + } + } + } + + test("write bucketed data without partitionBy") { + for (source <- Seq("parquet", "json", "orc")) { + withTable("bucketed_table") { + df.write + .format(source) + .bucketBy(8, "i", "j") + .saveAsTable("bucketed_table") + + val tableDir = new File(hiveContext.warehousePath, "bucketed_table") + testBucketing(tableDir, source, Seq("i", "j")) + } + } + } + + test("write bucketed data without partitionBy with sortBy") { + for (source <- Seq("parquet", "json", "orc")) { + withTable("bucketed_table") { + df.write + .format(source) + .bucketBy(8, "i", "j") + .sortBy("k") + .saveAsTable("bucketed_table") + + val tableDir = new File(hiveContext.warehousePath, "bucketed_table") + testBucketing(tableDir, source, Seq("i", "j"), Seq("k")) + } + } + } +} From ac56cf605b61803c26e0004b43c703cca7e02d61 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 6 Jan 2016 17:17:32 -0800 Subject: [PATCH 013/110] [SPARK-12604][CORE] Java count(AprroxDistinct)ByKey methods return Scala Long not Java Change Java countByKey, countApproxDistinctByKey return types to use Java Long, not Scala; update similar methods for consistency on java.long.Long.valueOf with no API change Author: Sean Owen Closes #10554 from srowen/SPARK-12604. --- .../apache/spark/api/java/JavaPairRDD.scala | 32 +++++++++++-------- .../apache/spark/api/java/JavaRDDLike.scala | 16 +++++----- .../java/org/apache/spark/JavaAPISuite.java | 18 +++++------ .../streaming/api/java/JavaDStreamLike.scala | 18 +++++------ .../streaming/api/java/JavaPairDStream.scala | 7 ++-- 5 files changed, 49 insertions(+), 42 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 91dc18697c352..76752e1fde663 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,8 +17,9 @@ package org.apache.spark.api.java +import java.{lang => jl} import java.lang.{Iterable => JIterable} -import java.util.{Comparator, List => JList, Map => JMap} +import java.util.{Comparator, List => JList} import scala.collection.JavaConverters._ import scala.language.implicitConversions @@ -139,7 +140,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * math.ceil(numItems * samplingRate) over all key values. */ def sampleByKey(withReplacement: Boolean, - fractions: JMap[K, Double], + fractions: java.util.Map[K, Double], seed: Long): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.sampleByKey(withReplacement, fractions.asScala, seed)) @@ -154,7 +155,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Use Utils.random.nextLong as the default seed for the random number generator. */ def sampleByKey(withReplacement: Boolean, - fractions: JMap[K, Double]): JavaPairRDD[K, V] = + fractions: java.util.Map[K, Double]): JavaPairRDD[K, V] = sampleByKey(withReplacement, fractions, Utils.random.nextLong) /** @@ -168,7 +169,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * two additional passes. */ def sampleByKeyExact(withReplacement: Boolean, - fractions: JMap[K, Double], + fractions: java.util.Map[K, Double], seed: Long): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.sampleByKeyExact(withReplacement, fractions.asScala, seed)) @@ -184,7 +185,9 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * * Use Utils.random.nextLong as the default seed for the random number generator. */ - def sampleByKeyExact(withReplacement: Boolean, fractions: JMap[K, Double]): JavaPairRDD[K, V] = + def sampleByKeyExact( + withReplacement: Boolean, + fractions: java.util.Map[K, Double]): JavaPairRDD[K, V] = sampleByKeyExact(withReplacement, fractions, Utils.random.nextLong) /** @@ -292,7 +295,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) 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] = mapAsSerializableJavaMap(rdd.countByKey()) + def countByKey(): java.util.Map[K, jl.Long] = + mapAsSerializableJavaMap(rdd.countByKey().mapValues(jl.Long.valueOf)) /** * Approximate version of countByKey that can return a partial result if it does @@ -934,9 +938,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * It must be greater than 0.000017. * @param partitioner partitioner of the resulting RDD. */ - def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): JavaPairRDD[K, Long] = - { - fromRDD(rdd.countApproxDistinctByKey(relativeSD, partitioner)) + def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner) + : JavaPairRDD[K, jl.Long] = { + fromRDD(rdd.countApproxDistinctByKey(relativeSD, partitioner)). + asInstanceOf[JavaPairRDD[K, jl.Long]] } /** @@ -950,8 +955,9 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * It must be greater than 0.000017. * @param numPartitions number of partitions of the resulting RDD. */ - def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): JavaPairRDD[K, Long] = { - fromRDD(rdd.countApproxDistinctByKey(relativeSD, numPartitions)) + def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): JavaPairRDD[K, jl.Long] = { + fromRDD(rdd.countApproxDistinctByKey(relativeSD, numPartitions)). + asInstanceOf[JavaPairRDD[K, jl.Long]] } /** @@ -964,8 +970,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * @param relativeSD Relative accuracy. Smaller values create counters that require more space. * It must be greater than 0.000017. */ - def countApproxDistinctByKey(relativeSD: Double): JavaPairRDD[K, Long] = { - fromRDD(rdd.countApproxDistinctByKey(relativeSD)) + def countApproxDistinctByKey(relativeSD: Double): JavaPairRDD[K, jl.Long] = { + fromRDD(rdd.countApproxDistinctByKey(relativeSD)).asInstanceOf[JavaPairRDD[K, jl.Long]] } /** Assign a name to this RDD */ 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 6d3485d88a163..1b1a9dce397fd 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -18,7 +18,7 @@ package org.apache.spark.api.java import java.{lang => jl} -import java.lang.{Iterable => JIterable, Long => JLong} +import java.lang.{Iterable => JIterable} import java.util.{Comparator, Iterator => JIterator, List => JList} import scala.collection.JavaConverters._ @@ -305,8 +305,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * 2*n+k, ..., where n is the number of partitions. So there may exist gaps, but this method * won't trigger a spark job, which is different from [[org.apache.spark.rdd.RDD#zipWithIndex]]. */ - def zipWithUniqueId(): JavaPairRDD[T, JLong] = { - JavaPairRDD.fromRDD(rdd.zipWithUniqueId()).asInstanceOf[JavaPairRDD[T, JLong]] + def zipWithUniqueId(): JavaPairRDD[T, jl.Long] = { + JavaPairRDD.fromRDD(rdd.zipWithUniqueId()).asInstanceOf[JavaPairRDD[T, jl.Long]] } /** @@ -316,8 +316,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * This is similar to Scala's zipWithIndex but it uses Long instead of Int as the index type. * This method needs to trigger a spark job when this RDD contains more than one partitions. */ - def zipWithIndex(): JavaPairRDD[T, JLong] = { - JavaPairRDD.fromRDD(rdd.zipWithIndex()).asInstanceOf[JavaPairRDD[T, JLong]] + def zipWithIndex(): JavaPairRDD[T, jl.Long] = { + JavaPairRDD.fromRDD(rdd.zipWithIndex()).asInstanceOf[JavaPairRDD[T, jl.Long]] } // Actions (launch a job to return a value to the user program) @@ -448,7 +448,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, jl.Long] = - mapAsSerializableJavaMap(rdd.countByValue().map((x => (x._1, new jl.Long(x._2))))) + mapAsSerializableJavaMap(rdd.countByValue().mapValues(jl.Long.valueOf)) /** * (Experimental) Approximate version of countByValue(). @@ -631,8 +631,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * The asynchronous version of `count`, which returns a * future for counting the number of elements in this RDD. */ - def countAsync(): JavaFutureAction[JLong] = { - new JavaFutureActionWrapper[Long, JLong](rdd.countAsync(), JLong.valueOf) + def countAsync(): JavaFutureAction[jl.Long] = { + new JavaFutureActionWrapper[Long, jl.Long](rdd.countAsync(), jl.Long.valueOf) } /** diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 502f86f178fd2..47382e4231563 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -1580,11 +1580,11 @@ public void countApproxDistinctByKey() { } double relativeSD = 0.001; JavaPairRDD pairRdd = sc.parallelizePairs(arrayData); - List> res = pairRdd.countApproxDistinctByKey(relativeSD, 8).collect(); - for (Tuple2 resItem : res) { - double count = (double)resItem._1(); - Long resCount = (Long)resItem._2(); - Double error = Math.abs((resCount - count) / count); + List> res = pairRdd.countApproxDistinctByKey(relativeSD, 8).collect(); + for (Tuple2 resItem : res) { + double count = resItem._1(); + long resCount = resItem._2(); + double error = Math.abs((resCount - count) / count); Assert.assertTrue(error < 0.1); } @@ -1633,12 +1633,12 @@ public Tuple2 call(Integer i) { fractions.put(0, 0.5); fractions.put(1, 1.0); JavaPairRDD wr = rdd2.sampleByKey(true, fractions, 1L); - Map wrCounts = (Map) (Object) wr.countByKey(); + Map wrCounts = wr.countByKey(); Assert.assertEquals(2, wrCounts.size()); 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(); + Map worCounts = wor.countByKey(); Assert.assertEquals(2, worCounts.size()); Assert.assertTrue(worCounts.get(0) > 0); Assert.assertTrue(worCounts.get(1) > 0); @@ -1659,12 +1659,12 @@ public Tuple2 call(Integer i) { fractions.put(0, 0.5); fractions.put(1, 1.0); JavaPairRDD wrExact = rdd2.sampleByKeyExact(true, fractions, 1L); - Map wrExactCounts = (Map) (Object) wrExact.countByKey(); + Map wrExactCounts = wrExact.countByKey(); Assert.assertEquals(2, wrExactCounts.size()); Assert.assertTrue(wrExactCounts.get(0) == 2); Assert.assertTrue(wrExactCounts.get(1) == 4); JavaPairRDD worExact = rdd2.sampleByKeyExact(false, fractions, 1L); - Map worExactCounts = (Map) (Object) worExact.countByKey(); + Map worExactCounts = worExact.countByKey(); Assert.assertEquals(2, worExactCounts.size()); Assert.assertTrue(worExactCounts.get(0) == 2); Assert.assertTrue(worExactCounts.get(1) == 4); 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 84acec7d8e330..733147f63ea2e 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 @@ -17,7 +17,7 @@ package org.apache.spark.streaming.api.java -import java.lang.{Long => JLong} +import java.{lang => jl} import java.util.{List => JList} import scala.collection.JavaConverters._ @@ -50,8 +50,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T def wrapRDD(in: RDD[T]): R - implicit def scalaIntToJavaLong(in: DStream[Long]): JavaDStream[JLong] = { - in.map(new JLong(_)) + implicit def scalaIntToJavaLong(in: DStream[Long]): JavaDStream[jl.Long] = { + in.map(jl.Long.valueOf) } /** @@ -74,14 +74,14 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * Return a new DStream in which each RDD has a single element generated by counting each RDD * of this DStream. */ - def count(): JavaDStream[JLong] = dstream.count() + def count(): JavaDStream[jl.Long] = dstream.count() /** * Return a new DStream in which each RDD contains the counts of each distinct value in * each RDD of this DStream. Hash partitioning is used to generate the RDDs with * Spark's default number of partitions. */ - def countByValue(): JavaPairDStream[T, JLong] = { + def countByValue(): JavaPairDStream[T, jl.Long] = { JavaPairDStream.scalaToJavaLong(dstream.countByValue()) } @@ -91,7 +91,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * partitions. * @param numPartitions number of partitions of each RDD in the new DStream. */ - def countByValue(numPartitions: Int): JavaPairDStream[T, JLong] = { + def countByValue(numPartitions: Int): JavaPairDStream[T, jl.Long] = { JavaPairDStream.scalaToJavaLong(dstream.countByValue(numPartitions)) } @@ -101,7 +101,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * 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() */ - def countByWindow(windowDuration: Duration, slideDuration: Duration) : JavaDStream[JLong] = { + def countByWindow(windowDuration: Duration, slideDuration: Duration) : JavaDStream[jl.Long] = { dstream.countByWindow(windowDuration, slideDuration) } @@ -116,7 +116,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * DStream's batching interval */ def countByValueAndWindow(windowDuration: Duration, slideDuration: Duration) - : JavaPairDStream[T, JLong] = { + : JavaPairDStream[T, jl.Long] = { JavaPairDStream.scalaToJavaLong( dstream.countByValueAndWindow(windowDuration, slideDuration)) } @@ -133,7 +133,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * @param numPartitions number of partitions of each RDD in the new DStream. */ def countByValueAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int) - : JavaPairDStream[T, JLong] = { + : JavaPairDStream[T, jl.Long] = { JavaPairDStream.scalaToJavaLong( dstream.countByValueAndWindow(windowDuration, slideDuration, numPartitions)) } 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 2bf3ccec6bc55..af0d84b33224f 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 @@ -17,7 +17,8 @@ package org.apache.spark.streaming.api.java -import java.lang.{Iterable => JIterable, Long => JLong} +import java.{lang => jl} +import java.lang.{Iterable => JIterable} import java.util.{List => JList} import scala.collection.JavaConverters._ @@ -847,7 +848,7 @@ object JavaPairDStream { } def scalaToJavaLong[K: ClassTag](dstream: JavaPairDStream[K, Long]) - : JavaPairDStream[K, JLong] = { - DStream.toPairDStreamFunctions(dstream.dstream).mapValues(new JLong(_)) + : JavaPairDStream[K, jl.Long] = { + DStream.toPairDStreamFunctions(dstream.dstream).mapValues(jl.Long.valueOf) } } From a74d743cc7c52a78fa023fdd0d06847b7d48bf78 Mon Sep 17 00:00:00 2001 From: Nong Li Date: Wed, 6 Jan 2016 19:20:43 -0800 Subject: [PATCH 014/110] [SPARK-12640][SQL] Add simple benchmarking utility class and add Parquet scan benchmarks. [SPARK-12640][SQL] Add simple benchmarking utility class and add Parquet scan benchmarks. We've run benchmarks ad hoc to measure the scanner performance. We will continue to invest in this and it makes sense to get these benchmarks into code. This adds a simple benchmarking utility to do this. Author: Nong Li Author: Nong Closes #10589 from nongli/spark-12640. --- .../org/apache/spark/util/Benchmark.scala | 120 +++++++++++++ .../parquet/ParquetReadBenchmark.scala | 158 ++++++++++++++++++ 2 files changed, 278 insertions(+) create mode 100644 core/src/main/scala/org/apache/spark/util/Benchmark.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala diff --git a/core/src/main/scala/org/apache/spark/util/Benchmark.scala b/core/src/main/scala/org/apache/spark/util/Benchmark.scala new file mode 100644 index 0000000000000..457a1a05a1bf5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/Benchmark.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.util + +import scala.collection.mutable + +import org.apache.commons.lang3.SystemUtils + +/** + * Utility class to benchmark components. An example of how to use this is: + * val benchmark = new Benchmark("My Benchmark", valuesPerIteration) + * benchmark.addCase("V1")() + * benchmark.addCase("V2")() + * benchmark.run + * This will output the average time to run each function and the rate of each function. + * + * The benchmark function takes one argument that is the iteration that's being run. + * + * If outputPerIteration is true, the timing for each run will be printed to stdout. + */ +private[spark] class Benchmark( + name: String, valuesPerIteration: Long, + iters: Int = 5, + outputPerIteration: Boolean = false) { + val benchmarks = mutable.ArrayBuffer.empty[Benchmark.Case] + + def addCase(name: String)(f: Int => Unit): Unit = { + benchmarks += Benchmark.Case(name, f) + } + + /** + * Runs the benchmark and outputs the results to stdout. This should be copied and added as + * a comment with the benchmark. Although the results vary from machine to machine, it should + * provide some baseline. + */ + def run(): Unit = { + require(benchmarks.nonEmpty) + // scalastyle:off + println("Running benchmark: " + name) + + val results = benchmarks.map { c => + println(" Running case: " + c.name) + Benchmark.measure(valuesPerIteration, iters, outputPerIteration)(c.fn) + } + println + + val firstRate = results.head.avgRate + // The results are going to be processor specific so it is useful to include that. + println(Benchmark.getProcessorName()) + printf("%-24s %16s %16s %14s\n", name + ":", "Avg Time(ms)", "Avg Rate(M/s)", "Relative Rate") + println("-------------------------------------------------------------------------") + results.zip(benchmarks).foreach { r => + printf("%-24s %16s %16s %14s\n", + r._2.name, + "%10.2f" format r._1.avgMs, + "%10.2f" format r._1.avgRate, + "%6.2f X" format (r._1.avgRate / firstRate)) + } + println + // scalastyle:on + } +} + +private[spark] object Benchmark { + case class Case(name: String, fn: Int => Unit) + case class Result(avgMs: Double, avgRate: Double) + + /** + * This should return a user helpful processor information. Getting at this depends on the OS. + * This should return something like "Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz" + */ + def getProcessorName(): String = { + if (SystemUtils.IS_OS_MAC_OSX) { + Utils.executeAndGetOutput(Seq("/usr/sbin/sysctl", "-n", "machdep.cpu.brand_string")) + } else if (SystemUtils.IS_OS_LINUX) { + Utils.executeAndGetOutput(Seq("/usr/bin/grep", "-m", "1", "\"model name\"", "/proc/cpuinfo")) + } else { + System.getenv("PROCESSOR_IDENTIFIER") + } + } + + /** + * Runs a single function `f` for iters, returning the average time the function took and + * the rate of the function. + */ + def measure(num: Long, iters: Int, outputPerIteration: Boolean)(f: Int => Unit): Result = { + var totalTime = 0L + for (i <- 0 until iters + 1) { + val start = System.nanoTime() + + f(i) + + val end = System.nanoTime() + if (i != 0) totalTime += end - start + + if (outputPerIteration) { + // scalastyle:off + println(s"Iteration $i took ${(end - start) / 1000} microseconds") + // scalastyle:on + } + } + Result(totalTime.toDouble / 1000000 / iters, num * iters / (totalTime.toDouble / 1000)) + } +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala new file mode 100644 index 0000000000000..cab6abde6da23 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.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.execution.datasources.parquet + +import java.io.File + +import scala.collection.JavaConverters._ +import scala.util.Try + +import org.apache.spark.sql.{SQLConf, SQLContext} +import org.apache.spark.util.{Benchmark, Utils} +import org.apache.spark.{SparkConf, SparkContext} + +/** + * Benchmark to measure parquet read performance. + * To run this: + * spark-submit --class --jars + */ +object ParquetReadBenchmark { + val conf = new SparkConf() + conf.set("spark.sql.parquet.compression.codec", "snappy") + val sc = new SparkContext("local[1]", "test-sql-context", conf) + val sqlContext = new SQLContext(sc) + + def withTempPath(f: File => Unit): Unit = { + val path = Utils.createTempDir() + path.delete() + try f(path) finally Utils.deleteRecursively(path) + } + + def withTempTable(tableNames: String*)(f: => Unit): Unit = { + try f finally tableNames.foreach(sqlContext.dropTempTable) + } + + def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + val (keys, values) = pairs.unzip + val currentValues = keys.map(key => Try(sqlContext.conf.getConfString(key)).toOption) + (keys, values).zipped.foreach(sqlContext.conf.setConfString) + try f finally { + keys.zip(currentValues).foreach { + case (key, Some(value)) => sqlContext.conf.setConfString(key, value) + case (key, None) => sqlContext.conf.unsetConf(key) + } + } + } + + def intScanBenchmark(values: Int): Unit = { + withTempPath { dir => + sqlContext.range(values).write.parquet(dir.getCanonicalPath) + withTempTable("tempTable") { + sqlContext.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + val benchmark = new Benchmark("Single Int Column Scan", values) + + benchmark.addCase("SQL Parquet Reader") { iter => + sqlContext.sql("select sum(id) from tempTable").collect() + } + + benchmark.addCase("SQL Parquet MR") { iter => + withSQLConf(SQLConf.PARQUET_UNSAFE_ROW_RECORD_READER_ENABLED.key -> "false") { + sqlContext.sql("select sum(id) from tempTable").collect() + } + } + + val files = SpecificParquetRecordReaderBase.listDirectory(dir).toArray + benchmark.addCase("ParquetReader") { num => + var sum = 0L + files.map(_.asInstanceOf[String]).foreach { p => + val reader = new UnsafeRowParquetRecordReader + reader.initialize(p, ("id" :: Nil).asJava) + + while (reader.nextKeyValue()) { + val record = reader.getCurrentValue + if (!record.isNullAt(0)) sum += record.getInt(0) + } + reader.close() + }} + + /* + Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz + Single Int Column Scan: Avg Time(ms) Avg Rate(M/s) Relative Rate + ------------------------------------------------------------------------- + SQL Parquet Reader 1910.0 13.72 1.00 X + SQL Parquet MR 2330.0 11.25 0.82 X + ParquetReader 1252.6 20.93 1.52 X + */ + benchmark.run() + } + } + } + + def intStringScanBenchmark(values: Int): Unit = { + withTempPath { dir => + withTempTable("t1", "tempTable") { + sqlContext.range(values).registerTempTable("t1") + sqlContext.sql("select id as c1, cast(id as STRING) as c2 from t1") + .write.parquet(dir.getCanonicalPath) + sqlContext.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + + val benchmark = new Benchmark("Int and String Scan", values) + + benchmark.addCase("SQL Parquet Reader") { iter => + sqlContext.sql("select sum(c1), sum(length(c2)) from tempTable").collect + } + + benchmark.addCase("SQL Parquet MR") { iter => + withSQLConf(SQLConf.PARQUET_UNSAFE_ROW_RECORD_READER_ENABLED.key -> "false") { + sqlContext.sql("select sum(c1), sum(length(c2)) from tempTable").collect + } + } + + val files = SpecificParquetRecordReaderBase.listDirectory(dir).toArray + benchmark.addCase("ParquetReader") { num => + var sum1 = 0L + var sum2 = 0L + files.map(_.asInstanceOf[String]).foreach { p => + val reader = new UnsafeRowParquetRecordReader + reader.initialize(p, null) + while (reader.nextKeyValue()) { + val record = reader.getCurrentValue + if (!record.isNullAt(0)) sum1 += record.getInt(0) + if (!record.isNullAt(1)) sum2 += record.getUTF8String(1).numBytes() + } + reader.close() + } + } + + /* + Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz + Int and String Scan: Avg Time(ms) Avg Rate(M/s) Relative Rate + ------------------------------------------------------------------------- + SQL Parquet Reader 2245.6 7.00 1.00 X + SQL Parquet MR 2914.2 5.40 0.77 X + ParquetReader 1544.6 10.18 1.45 X + */ + benchmark.run() + } + } + } + + def main(args: Array[String]): Unit = { + intScanBenchmark(1024 * 1024 * 15) + intStringScanBenchmark(1024 * 1024 * 10) + } +} From 6b6d02be0d4e2ce562dddfb391b3302f79de8276 Mon Sep 17 00:00:00 2001 From: Robert Dodier Date: Wed, 6 Jan 2016 19:49:10 -0800 Subject: [PATCH 015/110] [SPARK-12663][MLLIB] More informative error message in MLUtils.loadLibSVMFile This PR contains 1 commit which resolves [SPARK-12663](https://issues.apache.org/jira/browse/SPARK-12663). For the record, I got a positive response from 2 people when I floated this idea on devspark.apache.org on 2015-10-23. [Link to archived discussion.](http://apache-spark-developers-list.1001551.n3.nabble.com/slightly-more-informative-error-message-in-MLUtils-loadLibSVMFile-td14764.html) Author: Robert Dodier Closes #10611 from robert-dodier/loadlibsvmfile-error-msg-branch. --- mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala | 3 ++- 1 file changed, 2 insertions(+), 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 89186de96988f..74e9271e40329 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 @@ -86,7 +86,8 @@ object MLUtils { val indicesLength = indices.length while (i < indicesLength) { val current = indices(i) - require(current > previous, "indices should be one-based and in ascending order" ) + require(current > previous, s"indices should be one-based and in ascending order;" + + " found current=$current, previous=$previous; line=\"$line\"") previous = current i += 1 } From 8e19c7663a067d55b32af68d62da42c7cd5d6009 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 6 Jan 2016 20:50:31 -0800 Subject: [PATCH 016/110] [SPARK-7689] Remove TTL-based metadata cleaning in Spark 2.0 This PR removes `spark.cleaner.ttl` and the associated TTL-based metadata cleaning code. Now that we have the `ContextCleaner` and a timer to trigger periodic GCs, I don't think that `spark.cleaner.ttl` is necessary anymore. The TTL-based cleaning isn't enabled by default, isn't included in our end-to-end tests, and has been a source of user confusion when it is misconfigured. If the TTL is set too low, data which is still being used may be evicted / deleted, leading to hard to diagnose bugs. For all of these reasons, I think that we should remove this functionality in Spark 2.0. Additional benefits of doing this include marginally reduced memory usage, since we no longer need to store timetsamps in hashmaps, and a handful fewer threads. Author: Josh Rosen Closes #10534 from JoshRosen/remove-ttl-based-cleaning. --- .../org/apache/spark/MapOutputTracker.scala | 25 +-- .../scala/org/apache/spark/SparkContext.scala | 21 +-- .../shuffle/FileShuffleBlockResolver.scala | 28 ++- .../apache/spark/storage/BlockManager.scala | 63 ++----- .../apache/spark/util/MetadataCleaner.scala | 110 ----------- .../spark/util/TimeStampedHashSet.scala | 86 --------- .../util/TimeStampedWeakValueHashMap.scala | 171 ------------------ .../spark/util/TimeStampedHashMapSuite.scala | 86 --------- docs/configuration.md | 11 -- .../apache/spark/streaming/Checkpoint.scala | 3 +- .../spark/streaming/dstream/DStream.scala | 14 +- .../streaming/StreamingContextSuite.scala | 19 +- 12 files changed, 48 insertions(+), 589 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala delete mode 100644 core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala delete mode 100644 core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 8670f705cdb7e..1b59beb8d6efd 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -18,7 +18,6 @@ package org.apache.spark import java.io._ -import java.util.Arrays import java.util.concurrent.ConcurrentHashMap import java.util.zip.{GZIPInputStream, GZIPOutputStream} @@ -267,8 +266,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } /** - * MapOutputTracker for the driver. This uses TimeStampedHashMap to keep track of map - * output information, which allows old output information based on a TTL. + * MapOutputTracker for the driver. */ private[spark] class MapOutputTrackerMaster(conf: SparkConf) extends MapOutputTracker(conf) { @@ -291,17 +289,10 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) // can be read locally, but may lead to more delay in scheduling if those locations are busy. private val REDUCER_PREF_LOCS_FRACTION = 0.2 - /** - * Timestamp based HashMap for storing mapStatuses and cached serialized statuses in the driver, - * so that statuses are dropped only by explicit de-registering or by TTL-based cleaning (if set). - * Other than these two scenarios, nothing should be dropped from this HashMap. - */ - protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]() - private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]() - - // For cleaning up TimeStampedHashMaps - private val metadataCleaner = - new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup, conf) + // HashMaps for storing mapStatuses and cached serialized statuses in the driver. + // Statuses are dropped only by explicit de-registering. + protected val mapStatuses = new ConcurrentHashMap[Int, Array[MapStatus]]().asScala + private val cachedSerializedStatuses = new ConcurrentHashMap[Int, Array[Byte]]().asScala def registerShuffle(shuffleId: Int, numMaps: Int) { if (mapStatuses.put(shuffleId, new Array[MapStatus](numMaps)).isDefined) { @@ -462,14 +453,8 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) sendTracker(StopMapOutputTracker) mapStatuses.clear() trackerEndpoint = null - metadataCleaner.cancel() cachedSerializedStatuses.clear() } - - private def cleanup(cleanupTime: Long) { - mapStatuses.clearOldValues(cleanupTime) - cachedSerializedStatuses.clearOldValues(cleanupTime) - } } /** diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 4a99c0b081d6a..98075cef112db 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 java.io._ import java.lang.reflect.Constructor import java.net.URI import java.util.{Arrays, Properties, UUID} +import java.util.concurrent.ConcurrentMap import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReference} import java.util.UUID.randomUUID @@ -32,6 +33,7 @@ import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} import scala.util.control.NonFatal +import com.google.common.collect.MapMaker import org.apache.commons.lang.SerializationUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -199,7 +201,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private var _eventLogDir: Option[URI] = None private var _eventLogCodec: Option[String] = None private var _env: SparkEnv = _ - private var _metadataCleaner: MetadataCleaner = _ private var _jobProgressListener: JobProgressListener = _ private var _statusTracker: SparkStatusTracker = _ private var _progressBar: Option[ConsoleProgressBar] = None @@ -271,8 +272,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private[spark] val addedJars = HashMap[String, Long]() // Keeps track of all persisted RDDs - private[spark] val persistentRdds = new TimeStampedWeakValueHashMap[Int, RDD[_]] - private[spark] def metadataCleaner: MetadataCleaner = _metadataCleaner + private[spark] val persistentRdds = { + val map: ConcurrentMap[Int, RDD[_]] = new MapMaker().weakValues().makeMap[Int, RDD[_]]() + map.asScala + } private[spark] def jobProgressListener: JobProgressListener = _jobProgressListener def statusTracker: SparkStatusTracker = _statusTracker @@ -439,8 +442,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli _conf.set("spark.repl.class.uri", replUri) } - _metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, _conf) - _statusTracker = new SparkStatusTracker(this) _progressBar = @@ -1674,11 +1675,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli env.metricsSystem.report() } } - if (metadataCleaner != null) { - Utils.tryLogNonFatalError { - metadataCleaner.cancel() - } - } Utils.tryLogNonFatalError { _cleaner.foreach(_.stop()) } @@ -2085,11 +2081,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } } - /** Called by MetadataCleaner to clean up the persistentRdds map periodically */ - private[spark] def cleanup(cleanupTime: Long) { - persistentRdds.clearOldValues(cleanupTime) - } - // In order to prevent multiple SparkContexts from being active at the same time, mark this // context as having finished construction. // NOTE: this must be placed at the end of the SparkContext constructor. diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala index 7abcb29672cf5..294e16cde1931 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -17,7 +17,7 @@ package org.apache.spark.shuffle -import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} import scala.collection.JavaConverters._ @@ -27,7 +27,7 @@ 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.storage._ -import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} +import org.apache.spark.util.Utils /** A group of writers for a ShuffleMapTask, one writer per reducer. */ private[spark] trait ShuffleWriterGroup { @@ -63,10 +63,7 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) val completedMapTasks = new ConcurrentLinkedQueue[Int]() } - private val shuffleStates = new TimeStampedHashMap[ShuffleId, ShuffleState] - - private val metadataCleaner = - new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup, conf) + private val shuffleStates = new ConcurrentHashMap[ShuffleId, ShuffleState] /** * Get a ShuffleWriterGroup for the given map task, which will register it as complete @@ -75,9 +72,12 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) def forMapTask(shuffleId: Int, mapId: Int, numReducers: Int, serializer: Serializer, writeMetrics: ShuffleWriteMetrics): ShuffleWriterGroup = { new ShuffleWriterGroup { - shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numReducers)) - private val shuffleState = shuffleStates(shuffleId) - + private val shuffleState: ShuffleState = { + // Note: we do _not_ want to just wrap this java ConcurrentHashMap into a Scala map and use + // .getOrElseUpdate() because that's actually NOT atomic. + shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numReducers)) + shuffleStates.get(shuffleId) + } val openStartTime = System.nanoTime val serializerInstance = serializer.newInstance() val writers: Array[DiskBlockObjectWriter] = { @@ -114,7 +114,7 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) /** Remove all the blocks / files related to a particular shuffle. */ private def removeShuffleBlocks(shuffleId: ShuffleId): Boolean = { - shuffleStates.get(shuffleId) match { + Option(shuffleStates.get(shuffleId)) match { case Some(state) => for (mapId <- state.completedMapTasks.asScala; reduceId <- 0 until state.numReducers) { val blockId = new ShuffleBlockId(shuffleId, mapId, reduceId) @@ -131,11 +131,5 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) } } - private def cleanup(cleanupTime: Long) { - shuffleStates.clearOldValues(cleanupTime, (shuffleId, state) => removeShuffleBlocks(shuffleId)) - } - - override def stop() { - metadataCleaner.cancel() - } + override def stop(): Unit = {} } 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 8caf9e55359e0..5c80ac17b8d90 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -19,7 +19,9 @@ package org.apache.spark.storage import java.io._ import java.nio.{ByteBuffer, MappedByteBuffer} +import java.util.concurrent.ConcurrentHashMap +import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.{Await, ExecutionContext, Future} import scala.concurrent.duration._ @@ -75,7 +77,7 @@ private[spark] class BlockManager( val diskBlockManager = new DiskBlockManager(this, conf) - private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] + private val blockInfo = new ConcurrentHashMap[BlockId, BlockInfo] private val futureExecutionContext = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("block-manager-future", 128)) @@ -147,11 +149,6 @@ private[spark] class BlockManager( private var asyncReregisterTask: Future[Unit] = null private val asyncReregisterLock = new Object - private val metadataCleaner = new MetadataCleaner( - MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks, conf) - 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 @@ -232,7 +229,7 @@ private[spark] class BlockManager( */ private def reportAllBlocks(): Unit = { logInfo(s"Reporting ${blockInfo.size} blocks to the master.") - for ((blockId, info) <- blockInfo) { + for ((blockId, info) <- blockInfo.asScala) { val status = getCurrentBlockStatus(blockId, info) if (!tryToReportBlockStatus(blockId, info, status)) { logError(s"Failed to report $blockId to master; giving up.") @@ -313,7 +310,7 @@ private[spark] class BlockManager( * NOTE: This is mainly for testing, and it doesn't fetch information from external block store. */ def getStatus(blockId: BlockId): Option[BlockStatus] = { - blockInfo.get(blockId).map { info => + blockInfo.asScala.get(blockId).map { info => val memSize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L val diskSize = if (diskStore.contains(blockId)) diskStore.getSize(blockId) else 0L // Assume that block is not in external block store @@ -327,7 +324,7 @@ private[spark] class BlockManager( * may not know of). */ def getMatchingBlockIds(filter: BlockId => Boolean): Seq[BlockId] = { - (blockInfo.keys ++ diskBlockManager.getAllBlocks()).filter(filter).toSeq + (blockInfo.asScala.keys ++ diskBlockManager.getAllBlocks()).filter(filter).toSeq } /** @@ -439,7 +436,7 @@ private[spark] class BlockManager( } private def doGetLocal(blockId: BlockId, asBlockResult: Boolean): Option[Any] = { - val info = blockInfo.get(blockId).orNull + val info = blockInfo.get(blockId) if (info != null) { info.synchronized { // Double check to make sure the block is still there. There is a small chance that the @@ -447,7 +444,7 @@ private[spark] class BlockManager( // Note that this only checks metadata tracking. If user intentionally deleted the block // on disk or from off heap storage without using removeBlock, this conditional check will // still pass but eventually we will get an exception because we can't find the block. - if (blockInfo.get(blockId).isEmpty) { + if (blockInfo.asScala.get(blockId).isEmpty) { logWarning(s"Block $blockId had been removed") return None } @@ -731,7 +728,7 @@ private[spark] class BlockManager( val putBlockInfo = { val tinfo = new BlockInfo(level, tellMaster) // Do atomically ! - val oldBlockOpt = blockInfo.putIfAbsent(blockId, tinfo) + val oldBlockOpt = Option(blockInfo.putIfAbsent(blockId, tinfo)) if (oldBlockOpt.isDefined) { if (oldBlockOpt.get.waitForReady()) { logWarning(s"Block $blockId already exists on this machine; not re-adding it") @@ -1032,7 +1029,7 @@ private[spark] class BlockManager( data: () => Either[Array[Any], ByteBuffer]): Option[BlockStatus] = { logInfo(s"Dropping block $blockId from memory") - val info = blockInfo.get(blockId).orNull + val info = blockInfo.get(blockId) // If the block has not already been dropped if (info != null) { @@ -1043,7 +1040,7 @@ private[spark] class BlockManager( // If we get here, the block write failed. logWarning(s"Block $blockId was marked as failure. Nothing to drop") return None - } else if (blockInfo.get(blockId).isEmpty) { + } else if (blockInfo.asScala.get(blockId).isEmpty) { logWarning(s"Block $blockId was already dropped.") return None } @@ -1095,7 +1092,7 @@ private[spark] class BlockManager( def removeRdd(rddId: Int): Int = { // TODO: Avoid a linear scan by creating another mapping of RDD.id to blocks. logInfo(s"Removing RDD $rddId") - val blocksToRemove = blockInfo.keys.flatMap(_.asRDDId).filter(_.rddId == rddId) + val blocksToRemove = blockInfo.asScala.keys.flatMap(_.asRDDId).filter(_.rddId == rddId) blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster = false) } blocksToRemove.size } @@ -1105,7 +1102,7 @@ private[spark] class BlockManager( */ def removeBroadcast(broadcastId: Long, tellMaster: Boolean): Int = { logDebug(s"Removing broadcast $broadcastId") - val blocksToRemove = blockInfo.keys.collect { + val blocksToRemove = blockInfo.asScala.keys.collect { case bid @ BroadcastBlockId(`broadcastId`, _) => bid } blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster) } @@ -1117,7 +1114,7 @@ private[spark] class BlockManager( */ def removeBlock(blockId: BlockId, tellMaster: Boolean = true): Unit = { logDebug(s"Removing block $blockId") - val info = blockInfo.get(blockId).orNull + val info = blockInfo.get(blockId) if (info != null) { info.synchronized { // Removals are idempotent in disk store and memory store. At worst, we get a warning. @@ -1141,36 +1138,6 @@ private[spark] class BlockManager( } } - private def dropOldNonBroadcastBlocks(cleanupTime: Long): Unit = { - logInfo(s"Dropping non broadcast blocks older than $cleanupTime") - dropOldBlocks(cleanupTime, !_.isBroadcast) - } - - private def dropOldBroadcastBlocks(cleanupTime: Long): Unit = { - logInfo(s"Dropping broadcast blocks older than $cleanupTime") - dropOldBlocks(cleanupTime, _.isBroadcast) - } - - private def dropOldBlocks(cleanupTime: Long, shouldDrop: (BlockId => Boolean)): Unit = { - val iterator = blockInfo.getEntrySet.iterator - while (iterator.hasNext) { - val entry = iterator.next() - val (id, info, time) = (entry.getKey, entry.getValue.value, entry.getValue.timestamp) - if (time < cleanupTime && shouldDrop(id)) { - info.synchronized { - val level = info.level - if (level.useMemory) { memoryStore.remove(id) } - if (level.useDisk) { diskStore.remove(id) } - if (level.useOffHeap) { externalBlockStore.remove(id) } - iterator.remove() - logInfo(s"Dropped block $id") - } - val status = getCurrentBlockStatus(id, info) - reportBlockStatus(id, info, status) - } - } - } - private def shouldCompress(blockId: BlockId): Boolean = { blockId match { case _: ShuffleBlockId => compressShuffle @@ -1248,8 +1215,6 @@ private[spark] class BlockManager( if (externalBlockStoreInitialized) { externalBlockStore.clear() } - metadataCleaner.cancel() - broadcastCleaner.cancel() futureExecutionContext.shutdownNow() logInfo("BlockManager stopped") } diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala deleted file mode 100644 index a8bbad086849e..0000000000000 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ /dev/null @@ -1,110 +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 java.util.{Timer, TimerTask} - -import org.apache.spark.{Logging, SparkConf} - -/** - * Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries) - */ -private[spark] class MetadataCleaner( - cleanerType: MetadataCleanerType.MetadataCleanerType, - cleanupFunc: (Long) => Unit, - conf: SparkConf) - extends Logging -{ - val name = cleanerType.toString - - private val delaySeconds = MetadataCleaner.getDelaySeconds(conf, cleanerType) - private val periodSeconds = math.max(10, delaySeconds / 10) - private val timer = new Timer(name + " cleanup timer", true) - - - private val task = new TimerTask { - override def run() { - try { - cleanupFunc(System.currentTimeMillis() - (delaySeconds * 1000)) - logInfo("Ran metadata cleaner for " + name) - } catch { - case e: Exception => logError("Error running cleanup task for " + name, e) - } - } - } - - if (delaySeconds > 0) { - logDebug( - "Starting metadata cleaner for " + name + " with delay of " + delaySeconds + " seconds " + - "and period of " + periodSeconds + " secs") - timer.schedule(task, delaySeconds * 1000, periodSeconds * 1000) - } - - def cancel() { - timer.cancel() - } -} - -private[spark] object MetadataCleanerType extends Enumeration { - - val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, BLOCK_MANAGER, - SHUFFLE_BLOCK_MANAGER, BROADCAST_VARS = Value - - type MetadataCleanerType = Value - - def systemProperty(which: MetadataCleanerType.MetadataCleanerType): String = { - "spark.cleaner.ttl." + which.toString - } -} - -// TODO: This mutates a Conf to set properties right now, which is kind of ugly when used in the -// initialization of StreamingContext. It's okay for users trying to configure stuff themselves. -private[spark] object MetadataCleaner { - def getDelaySeconds(conf: SparkConf): Int = { - conf.getTimeAsSeconds("spark.cleaner.ttl", "-1").toInt - } - - def getDelaySeconds( - conf: SparkConf, - cleanerType: MetadataCleanerType.MetadataCleanerType): Int = { - conf.get(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString).toInt - } - - def setDelaySeconds( - conf: SparkConf, - cleanerType: MetadataCleanerType.MetadataCleanerType, - delay: Int) { - conf.set(MetadataCleanerType.systemProperty(cleanerType), delay.toString) - } - - /** - * Set the default delay time (in seconds). - * @param conf SparkConf instance - * @param delay default delay time to set - * @param resetAll whether to reset all to default - */ - def setDelaySeconds(conf: SparkConf, delay: Int, resetAll: Boolean = true) { - conf.set("spark.cleaner.ttl", delay.toString) - if (resetAll) { - for (cleanerType <- MetadataCleanerType.values) { - System.clearProperty(MetadataCleanerType.systemProperty(cleanerType)) - } - } - } -} - diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala deleted file mode 100644 index 65efeb1f4c19c..0000000000000 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala +++ /dev/null @@ -1,86 +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 java.util.concurrent.ConcurrentHashMap - -import scala.collection.JavaConverters._ -import scala.collection.mutable.Set - -private[spark] class TimeStampedHashSet[A] extends Set[A] { - val internalMap = new ConcurrentHashMap[A, Long]() - - def contains(key: A): Boolean = { - internalMap.contains(key) - } - - def iterator: Iterator[A] = { - val jIterator = internalMap.entrySet().iterator() - jIterator.asScala.map(_.getKey) - } - - override def + (elem: A): Set[A] = { - val newSet = new TimeStampedHashSet[A] - newSet ++= this - newSet += elem - newSet - } - - override def - (elem: A): Set[A] = { - val newSet = new TimeStampedHashSet[A] - newSet ++= this - newSet -= elem - newSet - } - - override def += (key: A): this.type = { - internalMap.put(key, currentTime) - this - } - - override def -= (key: A): this.type = { - internalMap.remove(key) - this - } - - override def empty: Set[A] = new TimeStampedHashSet[A]() - - override def size(): Int = internalMap.size() - - override def foreach[U](f: (A) => U): Unit = { - val iterator = internalMap.entrySet().iterator() - while(iterator.hasNext) { - f(iterator.next.getKey) - } - } - - /** - * Removes old values that have timestamp earlier than `threshTime` - */ - def clearOldValues(threshTime: Long) { - val iterator = internalMap.entrySet().iterator() - while(iterator.hasNext) { - val entry = iterator.next() - if (entry.getValue < threshTime) { - iterator.remove() - } - } - } - - private def currentTime: Long = System.currentTimeMillis() -} diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala deleted file mode 100644 index 310c0c109416c..0000000000000 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala +++ /dev/null @@ -1,171 +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 java.lang.ref.WeakReference -import java.util.concurrent.atomic.AtomicInteger - -import scala.collection.mutable -import scala.language.implicitConversions - -import org.apache.spark.Logging - -/** - * A wrapper of TimeStampedHashMap that ensures the values are weakly referenced and timestamped. - * - * If the value is garbage collected and the weak reference is null, get() will return a - * non-existent value. These entries are removed from the map periodically (every N inserts), as - * their values are no longer strongly reachable. Further, key-value pairs whose timestamps are - * older than a particular threshold can be removed using the clearOldValues method. - * - * TimeStampedWeakValueHashMap exposes a scala.collection.mutable.Map interface, which allows it - * to be a drop-in replacement for Scala HashMaps. Internally, it uses a Java ConcurrentHashMap, - * so all operations on this HashMap are thread-safe. - * - * @param updateTimeStampOnGet Whether timestamp of a pair will be updated when it is accessed. - */ -private[spark] class TimeStampedWeakValueHashMap[A, B](updateTimeStampOnGet: Boolean = false) - extends mutable.Map[A, B]() with Logging { - - import TimeStampedWeakValueHashMap._ - - private val internalMap = new TimeStampedHashMap[A, WeakReference[B]](updateTimeStampOnGet) - private val insertCount = new AtomicInteger(0) - - /** Return a map consisting only of entries whose values are still strongly reachable. */ - private def nonNullReferenceMap = internalMap.filter { case (_, ref) => ref.get != null } - - def get(key: A): Option[B] = internalMap.get(key) - - def iterator: Iterator[(A, B)] = nonNullReferenceMap.iterator - - override def + [B1 >: B](kv: (A, B1)): mutable.Map[A, B1] = { - val newMap = new TimeStampedWeakValueHashMap[A, B1] - val oldMap = nonNullReferenceMap.asInstanceOf[mutable.Map[A, WeakReference[B1]]] - newMap.internalMap.putAll(oldMap.toMap) - newMap.internalMap += kv - newMap - } - - override def - (key: A): mutable.Map[A, B] = { - val newMap = new TimeStampedWeakValueHashMap[A, B] - newMap.internalMap.putAll(nonNullReferenceMap.toMap) - newMap.internalMap -= key - newMap - } - - override def += (kv: (A, B)): this.type = { - internalMap += kv - if (insertCount.incrementAndGet() % CLEAR_NULL_VALUES_INTERVAL == 0) { - clearNullValues() - } - this - } - - override def -= (key: A): this.type = { - internalMap -= key - this - } - - override def update(key: A, value: B): Unit = this += ((key, value)) - - override def apply(key: A): B = internalMap.apply(key) - - override def filter(p: ((A, B)) => Boolean): mutable.Map[A, B] = nonNullReferenceMap.filter(p) - - override def empty: mutable.Map[A, B] = new TimeStampedWeakValueHashMap[A, B]() - - override def size: Int = internalMap.size - - override def foreach[U](f: ((A, B)) => U): Unit = nonNullReferenceMap.foreach(f) - - def putIfAbsent(key: A, value: B): Option[B] = internalMap.putIfAbsent(key, value) - - def toMap: Map[A, B] = iterator.toMap - - /** Remove old key-value pairs with timestamps earlier than `threshTime`. */ - def clearOldValues(threshTime: Long): Unit = internalMap.clearOldValues(threshTime) - - /** Remove entries with values that are no longer strongly reachable. */ - def clearNullValues() { - val it = internalMap.getEntrySet.iterator - while (it.hasNext) { - val entry = it.next() - if (entry.getValue.value.get == null) { - logDebug("Removing key " + entry.getKey + " because it is no longer strongly reachable.") - it.remove() - } - } - } - - // For testing - - def getTimestamp(key: A): Option[Long] = { - internalMap.getTimeStampedValue(key).map(_.timestamp) - } - - def getReference(key: A): Option[WeakReference[B]] = { - internalMap.getTimeStampedValue(key).map(_.value) - } -} - -/** - * Helper methods for converting to and from WeakReferences. - */ -private object TimeStampedWeakValueHashMap { - - // Number of inserts after which entries with null references are removed - val CLEAR_NULL_VALUES_INTERVAL = 100 - - /* Implicit conversion methods to WeakReferences. */ - - implicit def toWeakReference[V](v: V): WeakReference[V] = new WeakReference[V](v) - - implicit def toWeakReferenceTuple[K, V](kv: (K, V)): (K, WeakReference[V]) = { - kv match { case (k, v) => (k, toWeakReference(v)) } - } - - implicit def toWeakReferenceFunction[K, V, R](p: ((K, V)) => R): ((K, WeakReference[V])) => R = { - (kv: (K, WeakReference[V])) => p(kv) - } - - /* Implicit conversion methods from WeakReferences. */ - - implicit def fromWeakReference[V](ref: WeakReference[V]): V = ref.get - - implicit def fromWeakReferenceOption[V](v: Option[WeakReference[V]]): Option[V] = { - v match { - case Some(ref) => Option(fromWeakReference(ref)) - case None => None - } - } - - implicit def fromWeakReferenceTuple[K, V](kv: (K, WeakReference[V])): (K, V) = { - kv match { case (k, v) => (k, fromWeakReference(v)) } - } - - implicit def fromWeakReferenceIterator[K, V]( - it: Iterator[(K, WeakReference[V])]): Iterator[(K, V)] = { - it.map(fromWeakReferenceTuple) - } - - implicit def fromWeakReferenceMap[K, V]( - map: mutable.Map[K, WeakReference[V]]) : mutable.Map[K, V] = { - mutable.Map(map.mapValues(fromWeakReference).toSeq: _*) - } -} diff --git a/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala index 9b3169026cda3..25fc15dd54d04 100644 --- a/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.util -import java.lang.ref.WeakReference - import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.util.Random @@ -34,10 +32,6 @@ class TimeStampedHashMapSuite extends SparkFunSuite { testMap(new TimeStampedHashMap[String, String]()) testMapThreadSafety(new TimeStampedHashMap[String, String]()) - // Test TimeStampedWeakValueHashMap basic functionality - testMap(new TimeStampedWeakValueHashMap[String, String]()) - testMapThreadSafety(new TimeStampedWeakValueHashMap[String, String]()) - test("TimeStampedHashMap - clearing by timestamp") { // clearing by insertion time val map = new TimeStampedHashMap[String, String](updateTimeStampOnGet = false) @@ -68,86 +62,6 @@ class TimeStampedHashMapSuite extends SparkFunSuite { assert(map1.get("k2").isDefined) } - test("TimeStampedWeakValueHashMap - clearing by timestamp") { - // clearing by insertion time - val map = new TimeStampedWeakValueHashMap[String, String](updateTimeStampOnGet = false) - map("k1") = "v1" - assert(map("k1") === "v1") - Thread.sleep(10) - val threshTime = System.currentTimeMillis - assert(map.getTimestamp("k1").isDefined) - assert(map.getTimestamp("k1").get < threshTime) - map.clearOldValues(threshTime) - assert(map.get("k1") === None) - - // clearing by modification time - val map1 = new TimeStampedWeakValueHashMap[String, String](updateTimeStampOnGet = true) - map1("k1") = "v1" - map1("k2") = "v2" - assert(map1("k1") === "v1") - Thread.sleep(10) - val threshTime1 = System.currentTimeMillis - Thread.sleep(10) - assert(map1("k2") === "v2") // access k2 to update its access time to > threshTime - assert(map1.getTimestamp("k1").isDefined) - assert(map1.getTimestamp("k1").get < threshTime1) - assert(map1.getTimestamp("k2").isDefined) - assert(map1.getTimestamp("k2").get >= threshTime1) - map1.clearOldValues(threshTime1) // should only clear k1 - assert(map1.get("k1") === None) - assert(map1.get("k2").isDefined) - } - - test("TimeStampedWeakValueHashMap - clearing weak references") { - var strongRef = new Object - val weakRef = new WeakReference(strongRef) - val map = new TimeStampedWeakValueHashMap[String, Object] - map("k1") = strongRef - map("k2") = "v2" - map("k3") = "v3" - val isEquals = map("k1") == strongRef - assert(isEquals) - - // clear strong reference to "k1" - strongRef = null - val startTime = System.currentTimeMillis - System.gc() // Make a best effort to run the garbage collection. It *usually* runs GC. - System.runFinalization() // Make a best effort to call finalizer on all cleaned objects. - while(System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { - System.gc() - System.runFinalization() - Thread.sleep(100) - } - assert(map.getReference("k1").isDefined) - val ref = map.getReference("k1").get - assert(ref.get === null) - assert(map.get("k1") === None) - - // operations should only display non-null entries - assert(map.iterator.forall { case (k, v) => k != "k1" }) - assert(map.filter { case (k, v) => k != "k2" }.size === 1) - assert(map.filter { case (k, v) => k != "k2" }.head._1 === "k3") - assert(map.toMap.size === 2) - assert(map.toMap.forall { case (k, v) => k != "k1" }) - val buffer = new ArrayBuffer[String] - map.foreach { case (k, v) => buffer += v.toString } - assert(buffer.size === 2) - assert(buffer.forall(_ != "k1")) - val plusMap = map + (("k4", "v4")) - assert(plusMap.size === 3) - assert(plusMap.forall { case (k, v) => k != "k1" }) - val minusMap = map - "k2" - assert(minusMap.size === 1) - assert(minusMap.head._1 == "k3") - - // clear null values - should only clear k1 - map.clearNullValues() - assert(map.getReference("k1") === None) - assert(map.get("k1") === None) - assert(map.get("k2").isDefined) - assert(map.get("k2").get === "v2") - } - /** Test basic operations of a Scala mutable Map. */ def testMap(hashMapConstructor: => mutable.Map[String, String]) { def newMap() = hashMapConstructor diff --git a/docs/configuration.md b/docs/configuration.md index 7d743d572b582..3ffc77dcc62e0 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -823,17 +823,6 @@ Apart from these, the following properties are also available, and may be useful too small, BlockManager might take a performance hit. - - spark.cleaner.ttl - (infinite) - - Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks - generated, etc.). Periodic cleanups will ensure that metadata older than this duration will be - forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in - case of Spark Streaming applications). Note that any RDD that persists in memory for more than - this duration will be cleared as well. - - spark.executor.cores 1 in YARN mode, all the available cores on the worker in standalone mode. 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 61b230ab6f98a..b186d297610e2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -27,8 +27,8 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.Utils import org.apache.spark.streaming.scheduler.JobGenerator -import org.apache.spark.util.{MetadataCleaner, Utils} private[streaming] class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) @@ -40,7 +40,6 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) val checkpointDir = ssc.checkpointDir val checkpointDuration = ssc.checkpointDuration val pendingTimes = ssc.scheduler.getPendingTimes().toArray - val delaySeconds = MetadataCleaner.getDelaySeconds(ssc.conf) val sparkConfPairs = ssc.conf.getAll def createSparkConf(): SparkConf = { 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 91a43e14a8b1b..c59348a89d34f 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 @@ -32,7 +32,7 @@ import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext.rddToFileName import org.apache.spark.streaming.scheduler.Job import org.apache.spark.streaming.ui.UIUtils -import org.apache.spark.util.{CallSite, MetadataCleaner, Utils} +import org.apache.spark.util.{CallSite, Utils} /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous @@ -271,18 +271,6 @@ abstract class DStream[T: ClassTag] ( checkpointDuration + "). Please set it to higher than " + checkpointDuration + "." ) - val metadataCleanerDelay = MetadataCleaner.getDelaySeconds(ssc.conf) - logInfo("metadataCleanupDelay = " + metadataCleanerDelay) - require( - metadataCleanerDelay < 0 || rememberDuration.milliseconds < metadataCleanerDelay * 1000, - "It seems you are doing some DStream window operation or setting a checkpoint interval " + - "which requires " + this.getClass.getSimpleName + " to remember generated RDDs for more " + - "than " + rememberDuration.milliseconds / 1000 + " seconds. But Spark's metadata cleanup" + - "delay is set to " + metadataCleanerDelay + " seconds, which is not sufficient. Please " + - "set the Java cleaner delay to more than " + - math.ceil(rememberDuration.milliseconds / 1000.0).toInt + " seconds." - ) - dependencies.foreach(_.validateAtStart()) logInfo("Slide time = " + slideDuration) 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 860fac29c0ee0..0ae4c45988032 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -81,9 +81,9 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo test("from conf with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10s") + myConf.set("spark.dummyTimeConfig", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) + assert(ssc.conf.getTimeAsSeconds("spark.dummyTimeConfig", "-1") === 10) } test("from existing SparkContext") { @@ -93,26 +93,27 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo test("from existing SparkContext with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10s") + myConf.set("spark.dummyTimeConfig", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) + assert(ssc.conf.getTimeAsSeconds("spark.dummyTimeConfig", "-1") === 10) } test("from checkpoint") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10s") + myConf.set("spark.dummyTimeConfig", "10s") val ssc1 = new StreamingContext(myConf, batchDuration) addInputStream(ssc1).register() ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) assert( Utils.timeStringAsSeconds(cp.sparkConfPairs - .toMap.getOrElse("spark.cleaner.ttl", "-1")) === 10) + .toMap.getOrElse("spark.dummyTimeConfig", "-1")) === 10) ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(newCp.createSparkConf().getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) + assert( + newCp.createSparkConf().getTimeAsSeconds("spark.dummyTimeConfig", "-1") === 10) ssc = new StreamingContext(null, newCp, null) - assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) + assert(ssc.conf.getTimeAsSeconds("spark.dummyTimeConfig", "-1") === 10) } test("checkPoint from conf") { @@ -288,7 +289,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo test("stop gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) - conf.set("spark.cleaner.ttl", "3600s") + conf.set("spark.dummyTimeConfig", "3600s") sc = new SparkContext(conf) for (i <- 1 to 4) { logInfo("==================================\n\n\n") From 174e72ceca41a6ac17ad05d50832ee9c561918c0 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 6 Jan 2016 21:28:29 -0800 Subject: [PATCH 017/110] [SPARK-12673][UI] Add missing uri prepending for job description Otherwise the url will be failed to proxy to the right one if in YARN mode. Here is the screenshot: ![screen shot 2016-01-06 at 5 28 26 pm](https://cloud.githubusercontent.com/assets/850797/12139632/bbe78ecc-b49c-11e5-8932-94e8b3622a09.png) Author: jerryshao Closes #10618 from jerryshao/SPARK-12673. --- .../main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index db9912bc817e8..451cd83b51ae7 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -224,10 +224,10 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") val formattedSubmissionTime = job.submissionTime.map(UIUtils.formatDate).getOrElse("Unknown") - val jobDescription = UIUtils.makeDescription(lastStageDescription, parent.basePath) + val basePathUri = UIUtils.prependBaseUri(parent.basePath) + val jobDescription = UIUtils.makeDescription(lastStageDescription, basePathUri) - val detailUrl = - "%s/jobs/job?id=%s".format(UIUtils.prependBaseUri(parent.basePath), job.jobId) + val detailUrl = "%s/jobs/job?id=%s".format(basePathUri, job.jobId) {job.jobId} {job.jobGroup.map(id => s"($id)").getOrElse("")} From b6738520374637347ab5ae6c801730cdb6b35daa Mon Sep 17 00:00:00 2001 From: Guillaume Poulin Date: Wed, 6 Jan 2016 21:34:46 -0800 Subject: [PATCH 018/110] [SPARK-12678][CORE] MapPartitionsRDD clearDependencies MapPartitionsRDD was keeping a reference to `prev` after a call to `clearDependencies` which could lead to memory leak. Author: Guillaume Poulin Closes #10623 from gpoulin/map_partition_deps. --- .../main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 4312d3a417759..e4587c96eae1c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -25,7 +25,7 @@ import org.apache.spark.{Partition, TaskContext} * An RDD that applies the provided function to every partition of the parent RDD. */ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( - prev: RDD[T], + var prev: RDD[T], f: (TaskContext, Int, Iterator[T]) => Iterator[U], // (TaskContext, partition index, iterator) preservesPartitioning: Boolean = false) extends RDD[U](prev) { @@ -36,4 +36,9 @@ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( override def compute(split: Partition, context: TaskContext): Iterator[U] = f(context, split.index, firstParent[T].iterator(split, context)) + + override def clearDependencies() { + super.clearDependencies() + prev = null + } } From e5cde7ab11a43334fa01b1bb8904da5c0774bc62 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 6 Jan 2016 22:03:31 -0800 Subject: [PATCH 019/110] Revert "[SPARK-12006][ML][PYTHON] Fix GMM failure if initialModel is not None" This reverts commit fcd013cf70e7890aa25a8fe3cb6c8b36bf0e1f04. Author: Yin Huai Closes #10632 from yhuai/pythonStyle. --- python/pyspark/mllib/clustering.py | 2 +- python/pyspark/mllib/tests.py | 12 ------------ 2 files changed, 1 insertion(+), 13 deletions(-) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 48daa87e82d13..c9e6f1dec6bf8 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -346,7 +346,7 @@ def train(cls, rdd, k, convergenceTol=1e-3, maxIterations=100, seed=None, initia if initialModel.k != k: raise Exception("Mismatched cluster count, initialModel.k = %s, however k = %s" % (initialModel.k, k)) - initialModelWeights = list(initialModel.weights) + initialModelWeights = initialModel.weights initialModelMu = [initialModel.gaussians[i].mu for i in range(initialModel.k)] initialModelSigma = [initialModel.gaussians[i].sigma for i in range(initialModel.k)] java_model = callMLlibFunc("trainGaussianMixtureModel", rdd.map(_convert_to_vector), diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 97fed7662ea90..6ed03e35828ed 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -475,18 +475,6 @@ def test_gmm_deterministic(self): for c1, c2 in zip(clusters1.weights, clusters2.weights): self.assertEqual(round(c1, 7), round(c2, 7)) - def test_gmm_with_initial_model(self): - from pyspark.mllib.clustering import GaussianMixture - data = self.sc.parallelize([ - (-10, -5), (-9, -4), (10, 5), (9, 4) - ]) - - gmm1 = GaussianMixture.train(data, 2, convergenceTol=0.001, - maxIterations=10, seed=63) - gmm2 = GaussianMixture.train(data, 2, convergenceTol=0.001, - maxIterations=10, seed=63, initialModel=gmm1) - self.assertAlmostEqual((gmm1.weights - gmm2.weights).sum(), 0.0) - def test_classification(self): from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes from pyspark.mllib.tree import DecisionTree, DecisionTreeModel, RandomForest,\ From 84e77a15df18ba3f1cc871a3c52c783b46e52369 Mon Sep 17 00:00:00 2001 From: zzcclp Date: Wed, 6 Jan 2016 23:06:21 -0800 Subject: [PATCH 020/110] [DOC] fix 'spark.memory.offHeap.enabled' default value to false modify 'spark.memory.offHeap.enabled' default value to false Author: zzcclp Closes #10633 from zzcclp/fix_spark.memory.offHeap.enabled_default_value. --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 3ffc77dcc62e0..6bd0658b3e056 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -750,7 +750,7 @@ Apart from these, the following properties are also available, and may be useful spark.memory.offHeap.enabled - true + false If true, Spark will attempt to use off-heap memory for certain operations. If off-heap memory use is enabled, then spark.memory.offHeap.size must be positive. From 6a1c864ab6ee3e869a16ffdbaf6fead21c7aac6d Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 6 Jan 2016 23:21:52 -0800 Subject: [PATCH 021/110] [SPARK-12295] [SQL] external spilling for window functions This PR manage the memory used by window functions (buffered rows), also enable external spilling. After this PR, we can run window functions on a partition with hundreds of millions of rows with only 1G. Author: Davies Liu Closes #10605 from davies/unsafe_window. --- .../unsafe/sort/UnsafeExternalSorter.java | 21 +- .../unsafe/sort/UnsafeInMemorySorter.java | 18 +- .../unsafe/sort/UnsafeSorterIterator.java | 2 + .../unsafe/sort/UnsafeSorterSpillMerger.java | 7 + .../unsafe/sort/UnsafeSorterSpillReader.java | 8 +- .../apache/spark/sql/execution/Window.scala | 314 +++++++++++++----- 6 files changed, 276 insertions(+), 94 deletions(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 77d0b70bb892e..68dc0c6d415f6 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -45,7 +45,9 @@ public final class UnsafeExternalSorter extends MemoryConsumer { private final Logger logger = LoggerFactory.getLogger(UnsafeExternalSorter.class); + @Nullable private final PrefixComparator prefixComparator; + @Nullable private final RecordComparator recordComparator; private final TaskMemoryManager taskMemoryManager; private final BlockManager blockManager; @@ -431,7 +433,11 @@ class SpillableIterator extends UnsafeSorterIterator { public SpillableIterator(UnsafeInMemorySorter.SortedIterator inMemIterator) { this.upstream = inMemIterator; - this.numRecords = inMemIterator.numRecordsLeft(); + this.numRecords = inMemIterator.getNumRecords(); + } + + public int getNumRecords() { + return numRecords; } public long spill() throws IOException { @@ -558,13 +564,23 @@ class ChainedIterator extends UnsafeSorterIterator { private final Queue iterators; private UnsafeSorterIterator current; + private int numRecords; public ChainedIterator(Queue iterators) { assert iterators.size() > 0; + this.numRecords = 0; + for (UnsafeSorterIterator iter: iterators) { + this.numRecords += iter.getNumRecords(); + } this.iterators = iterators; this.current = iterators.remove(); } + @Override + public int getNumRecords() { + return numRecords; + } + @Override public boolean hasNext() { while (!current.hasNext() && !iterators.isEmpty()) { @@ -575,6 +591,9 @@ public boolean hasNext() { @Override public void loadNext() throws IOException { + while (!current.hasNext() && !iterators.isEmpty()) { + current = iterators.remove(); + } current.loadNext(); } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index b7ab45675ee1e..f71b8d154cc24 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -19,6 +19,8 @@ import java.util.Comparator; +import org.apache.avro.reflect.Nullable; + import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.unsafe.Platform; @@ -66,7 +68,9 @@ public int compare(RecordPointerAndKeyPrefix r1, RecordPointerAndKeyPrefix r2) { private final MemoryConsumer consumer; private final TaskMemoryManager memoryManager; + @Nullable private final Sorter sorter; + @Nullable private final Comparator sortComparator; /** @@ -98,10 +102,11 @@ public UnsafeInMemorySorter( LongArray array) { this.consumer = consumer; this.memoryManager = memoryManager; - this.sorter = new Sorter<>(UnsafeSortDataFormat.INSTANCE); if (recordComparator != null) { + this.sorter = new Sorter<>(UnsafeSortDataFormat.INSTANCE); this.sortComparator = new SortComparator(recordComparator, prefixComparator, memoryManager); } else { + this.sorter = null; this.sortComparator = null; } this.array = array; @@ -190,12 +195,13 @@ public SortedIterator clone() { } @Override - public boolean hasNext() { - return position / 2 < numRecords; + public int getNumRecords() { + return numRecords; } - public int numRecordsLeft() { - return numRecords - position / 2; + @Override + public boolean hasNext() { + return position / 2 < numRecords; } @Override @@ -227,7 +233,7 @@ public void loadNext() { * {@code next()} will return the same mutable object. */ public SortedIterator getSortedIterator() { - if (sortComparator != null) { + if (sorter != null) { sorter.sort(array, 0, pos / 2, sortComparator); } return new SortedIterator(pos / 2); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterIterator.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterIterator.java index 16ac2e8d821ba..1b3167fcc250c 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterIterator.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterIterator.java @@ -32,4 +32,6 @@ public abstract class UnsafeSorterIterator { public abstract int getRecordLength(); public abstract long getKeyPrefix(); + + public abstract int getNumRecords(); } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java index 3874a9f9cbdb6..ceb59352af64b 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java @@ -23,6 +23,7 @@ final class UnsafeSorterSpillMerger { + private int numRecords = 0; private final PriorityQueue priorityQueue; public UnsafeSorterSpillMerger( @@ -59,6 +60,7 @@ public void addSpillIfNotEmpty(UnsafeSorterIterator spillReader) throws IOExcept // priorityQueue, we will have n extra empty records in the result of the UnsafeSorterIterator. spillReader.loadNext(); priorityQueue.add(spillReader); + numRecords += spillReader.getNumRecords(); } } @@ -67,6 +69,11 @@ public UnsafeSorterIterator getSortedIterator() throws IOException { private UnsafeSorterIterator spillReader; + @Override + public int getNumRecords() { + return numRecords; + } + @Override public boolean hasNext() { return !priorityQueue.isEmpty() || (spillReader != null && spillReader.hasNext()); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java index dcb13e6581e54..20ee1c8eb0c77 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java @@ -38,6 +38,7 @@ public final class UnsafeSorterSpillReader extends UnsafeSorterIterator implemen // Variables that change with every record read: private int recordLength; private long keyPrefix; + private int numRecords; private int numRecordsRemaining; private byte[] arr = new byte[1024 * 1024]; @@ -53,13 +54,18 @@ public UnsafeSorterSpillReader( try { this.in = blockManager.wrapForCompression(blockId, bs); this.din = new DataInputStream(this.in); - numRecordsRemaining = din.readInt(); + numRecords = numRecordsRemaining = din.readInt(); } catch (IOException e) { Closeables.close(bs, /* swallowIOException = */ true); throw e; } } + @Override + public int getNumRecords() { + return numRecords; + } + @Override public boolean hasNext() { return (numRecordsRemaining > 0); diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala index 89b17c82459f3..be885397a7d40 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution +import java.util + import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -26,6 +28,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.types.IntegerType +import org.apache.spark.util.collection.unsafe.sort.{UnsafeExternalSorter, UnsafeSorterIterator} +import org.apache.spark.{SparkEnv, TaskContext} /** * This class calculates and outputs (windowed) aggregates over the rows in a single (sorted) @@ -283,23 +287,26 @@ case class Window( val grouping = UnsafeProjection.create(partitionSpec, child.output) // Manage the stream and the grouping. - var nextRow: InternalRow = EmptyRow - var nextGroup: InternalRow = EmptyRow + var nextRow: UnsafeRow = null + var nextGroup: UnsafeRow = null var nextRowAvailable: Boolean = false private[this] def fetchNextRow() { nextRowAvailable = stream.hasNext if (nextRowAvailable) { - nextRow = stream.next() + nextRow = stream.next().asInstanceOf[UnsafeRow] nextGroup = grouping(nextRow) } else { - nextRow = EmptyRow - nextGroup = EmptyRow + nextRow = null + nextGroup = null } } fetchNextRow() // Manage the current partition. - val rows = ArrayBuffer.empty[InternalRow] + val rows = ArrayBuffer.empty[UnsafeRow] + val inputFields = child.output.length + var sorter: UnsafeExternalSorter = null + var rowBuffer: RowBuffer = null val windowFunctionResult = new SpecificMutableRow(expressions.map(_.dataType)) val frames = factories.map(_(windowFunctionResult)) val numFrames = frames.length @@ -307,27 +314,63 @@ case class Window( // Collect all the rows in the current partition. // Before we start to fetch new input rows, make a copy of nextGroup. val currentGroup = nextGroup.copy() - rows.clear() + + // clear last partition + if (sorter != null) { + // the last sorter of this task will be cleaned up via task completion listener + sorter.cleanupResources() + sorter = null + } else { + rows.clear() + } + while (nextRowAvailable && nextGroup == currentGroup) { - rows += nextRow.copy() + if (sorter == null) { + rows += nextRow.copy() + + if (rows.length >= 4096) { + // We will not sort the rows, so prefixComparator and recordComparator are null. + sorter = UnsafeExternalSorter.create( + TaskContext.get().taskMemoryManager(), + SparkEnv.get.blockManager, + TaskContext.get(), + null, + null, + 1024, + SparkEnv.get.memoryManager.pageSizeBytes) + rows.foreach { r => + sorter.insertRecord(r.getBaseObject, r.getBaseOffset, r.getSizeInBytes, 0) + } + rows.clear() + } + } else { + sorter.insertRecord(nextRow.getBaseObject, nextRow.getBaseOffset, + nextRow.getSizeInBytes, 0) + } fetchNextRow() } + if (sorter != null) { + rowBuffer = new ExternalRowBuffer(sorter, inputFields) + } else { + rowBuffer = new ArrayRowBuffer(rows) + } // Setup the frames. var i = 0 while (i < numFrames) { - frames(i).prepare(rows) + frames(i).prepare(rowBuffer.copy()) i += 1 } // Setup iteration rowIndex = 0 - rowsSize = rows.size + rowsSize = rowBuffer.size() } // Iteration var rowIndex = 0 - var rowsSize = 0 + var rowsSize = 0L + override final def hasNext: Boolean = rowIndex < rowsSize || nextRowAvailable val join = new JoinedRow @@ -340,13 +383,14 @@ case class Window( if (rowIndex < rowsSize) { // Get the results for the window frames. var i = 0 + val current = rowBuffer.next() while (i < numFrames) { - frames(i).write() + frames(i).write(rowIndex, current) i += 1 } // 'Merge' the input row with the window function result - join(rows(rowIndex), windowFunctionResult) + join(current, windowFunctionResult) rowIndex += 1 // Return the projection. @@ -362,14 +406,18 @@ case class Window( * Function for comparing boundary values. */ private[execution] abstract class BoundOrdering { - def compare(input: Seq[InternalRow], inputIndex: Int, outputIndex: Int): Int + def compare(inputRow: InternalRow, inputIndex: Int, outputRow: InternalRow, outputIndex: Int): Int } /** * Compare the input index to the bound of the output index. */ private[execution] final case class RowBoundOrdering(offset: Int) extends BoundOrdering { - override def compare(input: Seq[InternalRow], inputIndex: Int, outputIndex: Int): Int = + override def compare( + inputRow: InternalRow, + inputIndex: Int, + outputRow: InternalRow, + outputIndex: Int): Int = inputIndex - (outputIndex + offset) } @@ -380,8 +428,100 @@ private[execution] final case class RangeBoundOrdering( ordering: Ordering[InternalRow], current: Projection, bound: Projection) extends BoundOrdering { - override def compare(input: Seq[InternalRow], inputIndex: Int, outputIndex: Int): Int = - ordering.compare(current(input(inputIndex)), bound(input(outputIndex))) + override def compare( + inputRow: InternalRow, + inputIndex: Int, + outputRow: InternalRow, + outputIndex: Int): Int = + ordering.compare(current(inputRow), bound(outputRow)) +} + +/** + * The interface of row buffer for a partition + */ +private[execution] abstract class RowBuffer { + + /** Number of rows. */ + def size(): Int + + /** Return next row in the buffer, null if no more left. */ + def next(): InternalRow + + /** Skip the next `n` rows. */ + def skip(n: Int): Unit + + /** Return a new RowBuffer that has the same rows. */ + def copy(): RowBuffer +} + +/** + * A row buffer based on ArrayBuffer (the number of rows is limited) + */ +private[execution] class ArrayRowBuffer(buffer: ArrayBuffer[UnsafeRow]) extends RowBuffer { + + private[this] var cursor: Int = -1 + + /** Number of rows. */ + def size(): Int = buffer.length + + /** Return next row in the buffer, null if no more left. */ + def next(): InternalRow = { + cursor += 1 + if (cursor < buffer.length) { + buffer(cursor) + } else { + null + } + } + + /** Skip the next `n` rows. */ + def skip(n: Int): Unit = { + cursor += n + } + + /** Return a new RowBuffer that has the same rows. */ + def copy(): RowBuffer = { + new ArrayRowBuffer(buffer) + } +} + +/** + * An external buffer of rows based on UnsafeExternalSorter + */ +private[execution] class ExternalRowBuffer(sorter: UnsafeExternalSorter, numFields: Int) + extends RowBuffer { + + private[this] val iter: UnsafeSorterIterator = sorter.getIterator + + private[this] val currentRow = new UnsafeRow(numFields) + + /** Number of rows. */ + def size(): Int = iter.getNumRecords() + + /** Return next row in the buffer, null if no more left. */ + def next(): InternalRow = { + if (iter.hasNext) { + iter.loadNext() + currentRow.pointTo(iter.getBaseObject, iter.getBaseOffset, iter.getRecordLength) + currentRow + } else { + null + } + } + + /** Skip the next `n` rows. */ + def skip(n: Int): Unit = { + var i = 0 + while (i < n && iter.hasNext) { + iter.loadNext() + i += 1 + } + } + + /** Return a new RowBuffer that has the same rows. */ + def copy(): RowBuffer = { + new ExternalRowBuffer(sorter, numFields) + } } /** @@ -395,12 +535,12 @@ private[execution] abstract class WindowFunctionFrame { * * @param rows to calculate the frame results for. */ - def prepare(rows: ArrayBuffer[InternalRow]): Unit + def prepare(rows: RowBuffer): Unit /** * Write the current results to the target row. */ - def write(): Unit + def write(index: Int, current: InternalRow): Unit } /** @@ -421,14 +561,11 @@ private[execution] final class OffsetWindowFunctionFrame( offset: Int) extends WindowFunctionFrame { /** Rows of the partition currently being processed. */ - private[this] var input: ArrayBuffer[InternalRow] = null + private[this] var input: RowBuffer = null /** Index of the input row currently used for output. */ private[this] var inputIndex = 0 - /** Index of the current output row. */ - private[this] var outputIndex = 0 - /** Row used when there is no valid input. */ private[this] val emptyRow = new GenericInternalRow(inputSchema.size) @@ -463,22 +600,26 @@ private[execution] final class OffsetWindowFunctionFrame( newMutableProjection(boundExpressions, Nil)().target(target) } - override def prepare(rows: ArrayBuffer[InternalRow]): Unit = { + override def prepare(rows: RowBuffer): Unit = { input = rows + // drain the first few rows if offset is larger than zero + inputIndex = 0 + while (inputIndex < offset) { + input.next() + inputIndex += 1 + } inputIndex = offset - outputIndex = 0 } - override def write(): Unit = { - val size = input.size - if (inputIndex >= 0 && inputIndex < size) { - join(input(inputIndex), input(outputIndex)) + override def write(index: Int, current: InternalRow): Unit = { + if (inputIndex >= 0 && inputIndex < input.size) { + val r = input.next() + join(r, current) } else { - join(emptyRow, input(outputIndex)) + join(emptyRow, current) } projection(join) inputIndex += 1 - outputIndex += 1 } } @@ -498,7 +639,13 @@ private[execution] final class SlidingWindowFunctionFrame( ubound: BoundOrdering) extends WindowFunctionFrame { /** Rows of the partition currently being processed. */ - private[this] var input: ArrayBuffer[InternalRow] = null + private[this] var input: RowBuffer = null + + /** The next row from `input`. */ + private[this] var nextRow: InternalRow = null + + /** The rows within current sliding window. */ + private[this] val buffer = new util.ArrayDeque[InternalRow]() /** Index of the first input row with a value greater than the upper bound of the current * output row. */ @@ -508,33 +655,32 @@ private[execution] final class SlidingWindowFunctionFrame( * current output row. */ private[this] var inputLowIndex = 0 - /** Index of the row we are currently writing. */ - private[this] var outputIndex = 0 - /** Prepare the frame for calculating a new partition. Reset all variables. */ - override def prepare(rows: ArrayBuffer[InternalRow]): Unit = { + override def prepare(rows: RowBuffer): Unit = { input = rows + nextRow = rows.next() inputHighIndex = 0 inputLowIndex = 0 - outputIndex = 0 + buffer.clear() } /** Write the frame columns for the current row to the given target row. */ - override def write(): Unit = { - var bufferUpdated = outputIndex == 0 + override def write(index: Int, current: InternalRow): Unit = { + var bufferUpdated = index == 0 // Add all rows to the buffer for which the input row value is equal to or less than // the output row upper bound. - while (inputHighIndex < input.size && - ubound.compare(input, inputHighIndex, outputIndex) <= 0) { + while (nextRow != null && ubound.compare(nextRow, inputHighIndex, current, index) <= 0) { + buffer.add(nextRow.copy()) + nextRow = input.next() inputHighIndex += 1 bufferUpdated = true } // Drop all rows from the buffer for which the input row value is smaller than // the output row lower bound. - while (inputLowIndex < inputHighIndex && - lbound.compare(input, inputLowIndex, outputIndex) < 0) { + while (!buffer.isEmpty && lbound.compare(buffer.peek(), inputLowIndex, current, index) < 0) { + buffer.remove() inputLowIndex += 1 bufferUpdated = true } @@ -542,12 +688,12 @@ private[execution] final class SlidingWindowFunctionFrame( // Only recalculate and update when the buffer changes. if (bufferUpdated) { processor.initialize(input.size) - processor.update(input, inputLowIndex, inputHighIndex) + val iter = buffer.iterator() + while (iter.hasNext) { + processor.update(iter.next()) + } processor.evaluate(target) } - - // Move to the next row. - outputIndex += 1 } } @@ -567,13 +713,18 @@ private[execution] final class UnboundedWindowFunctionFrame( processor: AggregateProcessor) extends WindowFunctionFrame { /** Prepare the frame for calculating a new partition. Process all rows eagerly. */ - override def prepare(rows: ArrayBuffer[InternalRow]): Unit = { - processor.initialize(rows.size) - processor.update(rows, 0, rows.size) + override def prepare(rows: RowBuffer): Unit = { + val size = rows.size() + processor.initialize(size) + var i = 0 + while (i < size) { + processor.update(rows.next()) + i += 1 + } } /** Write the frame columns for the current row to the given target row. */ - override def write(): Unit = { + override def write(index: Int, current: InternalRow): Unit = { // Unfortunately we cannot assume that evaluation is deterministic. So we need to re-evaluate // for each row. processor.evaluate(target) @@ -600,31 +751,32 @@ private[execution] final class UnboundedPrecedingWindowFunctionFrame( ubound: BoundOrdering) extends WindowFunctionFrame { /** Rows of the partition currently being processed. */ - private[this] var input: ArrayBuffer[InternalRow] = null + private[this] var input: RowBuffer = null + + /** The next row from `input`. */ + private[this] var nextRow: InternalRow = null /** Index of the first input row with a value greater than the upper bound of the current * output row. */ private[this] var inputIndex = 0 - /** Index of the row we are currently writing. */ - private[this] var outputIndex = 0 - /** Prepare the frame for calculating a new partition. */ - override def prepare(rows: ArrayBuffer[InternalRow]): Unit = { + override def prepare(rows: RowBuffer): Unit = { input = rows + nextRow = rows.next() inputIndex = 0 - outputIndex = 0 processor.initialize(input.size) } /** Write the frame columns for the current row to the given target row. */ - override def write(): Unit = { - var bufferUpdated = outputIndex == 0 + override def write(index: Int, current: InternalRow): Unit = { + var bufferUpdated = index == 0 // Add all rows to the aggregates for which the input row value is equal to or less than // the output row upper bound. - while (inputIndex < input.size && ubound.compare(input, inputIndex, outputIndex) <= 0) { - processor.update(input(inputIndex)) + while (nextRow != null && ubound.compare(nextRow, inputIndex, current, index) <= 0) { + processor.update(nextRow) + nextRow = input.next() inputIndex += 1 bufferUpdated = true } @@ -633,9 +785,6 @@ private[execution] final class UnboundedPrecedingWindowFunctionFrame( if (bufferUpdated) { processor.evaluate(target) } - - // Move to the next row. - outputIndex += 1 } } @@ -661,29 +810,31 @@ private[execution] final class UnboundedFollowingWindowFunctionFrame( lbound: BoundOrdering) extends WindowFunctionFrame { /** Rows of the partition currently being processed. */ - private[this] var input: ArrayBuffer[InternalRow] = null + private[this] var input: RowBuffer = null /** Index of the first input row with a value equal to or greater than the lower bound of the * current output row. */ private[this] var inputIndex = 0 - /** Index of the row we are currently writing. */ - private[this] var outputIndex = 0 - /** Prepare the frame for calculating a new partition. */ - override def prepare(rows: ArrayBuffer[InternalRow]): Unit = { + override def prepare(rows: RowBuffer): Unit = { input = rows inputIndex = 0 - outputIndex = 0 } /** Write the frame columns for the current row to the given target row. */ - override def write(): Unit = { - var bufferUpdated = outputIndex == 0 + override def write(index: Int, current: InternalRow): Unit = { + var bufferUpdated = index == 0 + + // Duplicate the input to have a new iterator + val tmp = input.copy() // Drop all rows from the buffer for which the input row value is smaller than // the output row lower bound. - while (inputIndex < input.size && lbound.compare(input, inputIndex, outputIndex) < 0) { + tmp.skip(inputIndex) + var nextRow = tmp.next() + while (nextRow != null && lbound.compare(nextRow, inputIndex, current, index) < 0) { + nextRow = tmp.next() inputIndex += 1 bufferUpdated = true } @@ -691,12 +842,12 @@ private[execution] final class UnboundedFollowingWindowFunctionFrame( // Only recalculate and update when the buffer changes. if (bufferUpdated) { processor.initialize(input.size) - processor.update(input, inputIndex, input.size) + while (nextRow != null) { + processor.update(nextRow) + nextRow = tmp.next() + } processor.evaluate(target) } - - // Move to the next row. - outputIndex += 1 } } @@ -825,15 +976,6 @@ private[execution] final class AggregateProcessor( } } - /** Bulk update the given buffer. */ - def update(input: ArrayBuffer[InternalRow], begin: Int, end: Int): Unit = { - var i = begin - while (i < end) { - update(input(i)) - i += 1 - } - } - /** Evaluate buffer. */ def evaluate(target: MutableRow): Unit = evaluateProjection.target(target)(buffer) From fd1dcfaf2608c2cc3a439ed3ca044ae655982306 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 6 Jan 2016 23:46:12 -0800 Subject: [PATCH 022/110] [SPARK-12542][SQL] support except/intersect in HiveQl Parse the SQL query with except/intersect in FROM clause for HivQL. Author: Davies Liu Closes #10622 from davies/intersect. --- .../spark/sql/catalyst/parser/SparkSqlLexer.g | 1 + .../sql/catalyst/parser/SparkSqlParser.g | 12 ++++--- .../spark/sql/catalyst/CatalystQl.scala | 7 +++- .../spark/sql/catalyst/CatalystQlSuite.scala | 32 +++++++++++++++++++ .../sql/hive/execution/HiveQuerySuite.scala | 18 +++++++++++ 5 files changed, 65 insertions(+), 5 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlLexer.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlLexer.g index e01e7101d0b7e..44a63fbef258c 100644 --- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlLexer.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlLexer.g @@ -103,6 +103,7 @@ KW_CLUSTER: 'CLUSTER'; KW_DISTRIBUTE: 'DISTRIBUTE'; KW_SORT: 'SORT'; KW_UNION: 'UNION'; +KW_EXCEPT: 'EXCEPT'; KW_LOAD: 'LOAD'; KW_EXPORT: 'EXPORT'; KW_IMPORT: 'IMPORT'; diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g index 4afce3090f739..cf8a56566d32d 100644 --- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g @@ -88,6 +88,8 @@ TOK_DISTRIBUTEBY; TOK_SORTBY; TOK_UNIONALL; TOK_UNIONDISTINCT; +TOK_EXCEPT; +TOK_INTERSECT; TOK_JOIN; TOK_LEFTOUTERJOIN; TOK_RIGHTOUTERJOIN; @@ -2122,6 +2124,8 @@ setOperator @after { popMsg(state); } : KW_UNION KW_ALL -> ^(TOK_UNIONALL) | KW_UNION KW_DISTINCT? -> ^(TOK_UNIONDISTINCT) + | KW_EXCEPT -> ^(TOK_EXCEPT) + | KW_INTERSECT -> ^(TOK_INTERSECT) ; queryStatementExpression[boolean topLevel] @@ -2242,7 +2246,7 @@ setOpSelectStatement[CommonTree t, boolean topLevel] ^(TOK_QUERY ^(TOK_FROM ^(TOK_SUBQUERY - ^(TOK_UNIONALL {$setOpSelectStatement.tree} $b) + ^($u {$setOpSelectStatement.tree} $b) {adaptor.create(Identifier, generateUnionAlias())} ) ) @@ -2252,12 +2256,12 @@ setOpSelectStatement[CommonTree t, boolean topLevel] ) ) -> {$setOpSelectStatement.tree != null && $u.tree.getType()!=SparkSqlParser.TOK_UNIONDISTINCT}? - ^(TOK_UNIONALL {$setOpSelectStatement.tree} $b) + ^($u {$setOpSelectStatement.tree} $b) -> {$setOpSelectStatement.tree == null && $u.tree.getType()==SparkSqlParser.TOK_UNIONDISTINCT}? ^(TOK_QUERY ^(TOK_FROM ^(TOK_SUBQUERY - ^(TOK_UNIONALL {$t} $b) + ^($u {$t} $b) {adaptor.create(Identifier, generateUnionAlias())} ) ) @@ -2266,7 +2270,7 @@ setOpSelectStatement[CommonTree t, boolean topLevel] ^(TOK_SELECTDI ^(TOK_SELEXPR TOK_ALLCOLREF)) ) ) - -> ^(TOK_UNIONALL {$t} $b) + -> ^($u {$t} $b) )+ o=orderByClause? c=clusterByClause? diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala index 42bdf25b61ea5..1eda4a9a97644 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala @@ -399,9 +399,14 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C // return With plan if there is CTE cteRelations.map(With(query, _)).getOrElse(query) - // HIVE-9039 renamed TOK_UNION => TOK_UNIONALL while adding TOK_UNIONDISTINCT case Token("TOK_UNIONALL", left :: right :: Nil) => Union(nodeToPlan(left), nodeToPlan(right)) + case Token("TOK_UNIONDISTINCT", left :: right :: Nil) => + Distinct(Union(nodeToPlan(left), nodeToPlan(right))) + case Token("TOK_EXCEPT", left :: right :: Nil) => + Except(nodeToPlan(left), nodeToPlan(right)) + case Token("TOK_INTERSECT", left :: right :: Nil) => + Intersect(nodeToPlan(left), nodeToPlan(right)) case _ => noParseRule("Plan", node) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala new file mode 100644 index 0000000000000..0fee97fb0718c --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.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.catalyst + +import org.apache.spark.sql.catalyst.plans.PlanTest + +class CatalystQlSuite extends PlanTest { + + test("parse union/except/intersect") { + val paresr = new CatalystQl() + paresr.createPlan("select * from t1 union all select * from t2") + paresr.createPlan("select * from t1 union distinct select * from t2") + paresr.createPlan("select * from t1 union select * from t2") + paresr.createPlan("select * from t1 except select * from t2") + paresr.createPlan("select * from t1 intersect select * from t2") + } +} 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 98e22c2e2c1b0..fa99289b41971 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 @@ -787,6 +787,24 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { assert(sql("select key from src having key > 490").collect().size < 100) } + test("union/except/intersect") { + assertResult(Array(Row(1), Row(1))) { + sql("select 1 as a union all select 1 as a").collect() + } + assertResult(Array(Row(1))) { + sql("select 1 as a union distinct select 1 as a").collect() + } + assertResult(Array(Row(1))) { + sql("select 1 as a union select 1 as a").collect() + } + assertResult(Array()) { + sql("select 1 as a except select 1 as a").collect() + } + assertResult(Array(Row(1))) { + sql("select 1 as a intersect select 1 as a").collect() + } + } + test("SPARK-5383 alias for udfs with multi output columns") { assert( sql("select stack(2, key, value, key, value) as (a, b) from src limit 5") From 8113dbda0bd51fdbe20dbfad466b8d25304a01f4 Mon Sep 17 00:00:00 2001 From: Jacek Laskowski Date: Thu, 7 Jan 2016 00:27:13 -0800 Subject: [PATCH 023/110] [STREAMING][DOCS][EXAMPLES] Minor fixes Author: Jacek Laskowski Closes #10603 from jaceklaskowski/streaming-actor-custom-receiver. --- docs/streaming-custom-receivers.md | 8 ++++---- .../spark/examples/streaming/ActorWordCount.scala | 10 ++++------ 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index a75587a92adc7..97db865daa371 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -257,9 +257,9 @@ The following table summarizes the characteristics of both types of receivers ## Implementing and Using a Custom Actor-based Receiver -Custom [Akka Actors](http://doc.akka.io/docs/akka/2.2.4/scala/actors.html) can also be used to +Custom [Akka Actors](http://doc.akka.io/docs/akka/2.3.11/scala/actors.html) can also be used to receive data. The [`ActorHelper`](api/scala/index.html#org.apache.spark.streaming.receiver.ActorHelper) -trait can be applied on any Akka actor, which allows received data to be stored in Spark using +trait can be mixed in to any Akka actor, which allows received data to be stored in Spark using `store(...)` methods. The supervisor strategy of this actor can be configured to handle failures, etc. {% highlight scala %} @@ -273,8 +273,8 @@ class CustomActor extends Actor with ActorHelper { And a new input stream can be created with this custom actor as {% highlight scala %} -// Assuming ssc is the StreamingContext -val lines = ssc.actorStream[String](Props(new CustomActor()), "CustomReceiver") +val ssc: StreamingContext = ... +val lines = ssc.actorStream[String](Props[CustomActor], "CustomReceiver") {% endhighlight %} See [ActorWordCount.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala index 8b8dae0be6119..a47fb7b7d7906 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala @@ -62,15 +62,13 @@ class FeederActor extends Actor { }.start() def receive: Receive = { - case SubscribeReceiver(receiverActor: ActorRef) => println("received subscribe from %s".format(receiverActor.toString)) - receivers = LinkedList(receiverActor) ++ receivers + receivers = LinkedList(receiverActor) ++ receivers case UnsubscribeReceiver(receiverActor: ActorRef) => println("received unsubscribe from %s".format(receiverActor.toString)) - receivers = receivers.dropWhile(x => x eq receiverActor) - + receivers = receivers.dropWhile(x => x eq receiverActor) } } @@ -129,9 +127,9 @@ object FeederActor { * and describe the AkkaSystem that Spark Sample feeder is running on. * * To run this example locally, you may run Feeder Actor as - * `$ bin/run-example org.apache.spark.examples.streaming.FeederActor 127.0.1.1 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.FeederActor 127.0.0.1 9999` * and then run the example - * `$ bin/run-example org.apache.spark.examples.streaming.ActorWordCount 127.0.1.1 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.ActorWordCount 127.0.0.1 9999` */ object ActorWordCount { def main(args: Array[String]) { From 592f64985d0d58b4f6a0366bf975e04ca496bdbe Mon Sep 17 00:00:00 2001 From: zero323 Date: Thu, 7 Jan 2016 10:32:56 -0800 Subject: [PATCH 024/110] [SPARK-12006][ML][PYTHON] Fix GMM failure if initialModel is not None If initial model passed to GMM is not empty it causes net.razorvine.pickle.PickleException. It can be fixed by converting initialModel.weights to list. Author: zero323 Closes #10644 from zero323/SPARK-12006. --- python/pyspark/mllib/clustering.py | 2 +- python/pyspark/mllib/tests.py | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index c9e6f1dec6bf8..48daa87e82d13 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -346,7 +346,7 @@ def train(cls, rdd, k, convergenceTol=1e-3, maxIterations=100, seed=None, initia if initialModel.k != k: raise Exception("Mismatched cluster count, initialModel.k = %s, however k = %s" % (initialModel.k, k)) - initialModelWeights = initialModel.weights + initialModelWeights = list(initialModel.weights) initialModelMu = [initialModel.gaussians[i].mu for i in range(initialModel.k)] initialModelSigma = [initialModel.gaussians[i].sigma for i in range(initialModel.k)] java_model = callMLlibFunc("trainGaussianMixtureModel", rdd.map(_convert_to_vector), diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 6ed03e35828ed..3436a28b2974f 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -475,6 +475,18 @@ def test_gmm_deterministic(self): for c1, c2 in zip(clusters1.weights, clusters2.weights): self.assertEqual(round(c1, 7), round(c2, 7)) + def test_gmm_with_initial_model(self): + from pyspark.mllib.clustering import GaussianMixture + data = self.sc.parallelize([ + (-10, -5), (-9, -4), (10, 5), (9, 4) + ]) + + gmm1 = GaussianMixture.train(data, 2, convergenceTol=0.001, + maxIterations=10, seed=63) + gmm2 = GaussianMixture.train(data, 2, convergenceTol=0.001, + maxIterations=10, seed=63, initialModel=gmm1) + self.assertAlmostEqual((gmm1.weights - gmm2.weights).sum(), 0.0) + def test_classification(self): from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes from pyspark.mllib.tree import DecisionTree, DecisionTreeModel, RandomForest,\ From f194d9911a93fc3a78be820096d4836f22d09976 Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Thu, 7 Jan 2016 10:37:15 -0800 Subject: [PATCH 025/110] [SPARK-12662][SQL] Fix DataFrame.randomSplit to avoid creating overlapping splits https://issues.apache.org/jira/browse/SPARK-12662 cc yhuai Author: Sameer Agarwal Closes #10626 from sameeragarwal/randomsplit. --- .../org/apache/spark/sql/DataFrame.scala | 7 +++++- .../apache/spark/sql/DataFrameStatSuite.scala | 22 +++++++++++++++++++ 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 7cf2818590a78..60d2f05b8605b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -1062,10 +1062,15 @@ class DataFrame private[sql]( * @since 1.4.0 */ def randomSplit(weights: Array[Double], seed: Long): Array[DataFrame] = { + // It is possible that the underlying dataframe doesn't guarantee the ordering of rows in its + // constituent partitions each time a split is materialized which could result in + // overlapping splits. To prevent this, we explicitly sort each input partition to make the + // ordering deterministic. + val sorted = Sort(logicalPlan.output.map(SortOrder(_, Ascending)), global = false, logicalPlan) val sum = weights.sum val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) normalizedCumWeights.sliding(2).map { x => - new DataFrame(sqlContext, Sample(x(0), x(1), withReplacement = false, seed, logicalPlan)) + new DataFrame(sqlContext, Sample(x(0), x(1), withReplacement = false, seed, sorted)) }.toArray } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index b15af42caa3ab..63ad6c439a870 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -62,6 +62,28 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { } } + test("randomSplit on reordered partitions") { + // This test ensures that randomSplit does not create overlapping splits even when the + // underlying dataframe (such as the one below) doesn't guarantee a deterministic ordering of + // rows in each partition. + val data = + sparkContext.parallelize(1 to 600, 2).mapPartitions(scala.util.Random.shuffle(_)).toDF("id") + val splits = data.randomSplit(Array[Double](2, 3), seed = 1) + + assert(splits.length == 2, "wrong number of splits") + + // Verify that the splits span the entire dataset + assert(splits.flatMap(_.collect()).toSet == data.collect().toSet) + + // Verify that the splits don't overalap + assert(splits(0).intersect(splits(1)).collect().isEmpty) + + // Verify that the results are deterministic across multiple runs + val firstRun = splits.toSeq.map(_.collect().toSeq) + val secondRun = data.randomSplit(Array[Double](2, 3), seed = 1).toSeq.map(_.collect().toSeq) + assert(firstRun == secondRun) + } + test("pearson correlation") { val df = Seq.tabulate(10)(i => (i, 2 * i, i * -1.0)).toDF("a", "b", "c") val corr1 = df.stat.corr("a", "b", "pearson") From 07b314a57a638a232ee0b5cd14169e57d742f0f9 Mon Sep 17 00:00:00 2001 From: Jacek Laskowski Date: Thu, 7 Jan 2016 10:39:46 -0800 Subject: [PATCH 026/110] [MINOR] Fix for BUILD FAILURE for Scala 2.11 It was introduced in 917d3fc069fb9ea1c1487119c9c12b373f4f9b77 /cc cloud-fan rxin Author: Jacek Laskowski Closes #10636 from jaceklaskowski/fix-for-build-failure-2.11. --- .../datasources/json/JSONRelation.scala | 19 +------------------ 1 file changed, 1 insertion(+), 18 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala index b92edf65bfb6b..8a6fa4aeebc09 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala @@ -68,29 +68,12 @@ private[sql] class JSONRelation( val maybeDataSchema: Option[StructType], val maybePartitionSpec: Option[PartitionSpec], override val userDefinedPartitionColumns: Option[StructType], - override val bucketSpec: Option[BucketSpec], + override val bucketSpec: Option[BucketSpec] = None, override val paths: Array[String] = Array.empty[String], parameters: Map[String, String] = Map.empty[String, String]) (@transient val sqlContext: SQLContext) extends HadoopFsRelation(maybePartitionSpec, parameters) { - def this( - inputRDD: Option[RDD[String]], - maybeDataSchema: Option[StructType], - maybePartitionSpec: Option[PartitionSpec], - userDefinedPartitionColumns: Option[StructType], - paths: Array[String] = Array.empty[String], - parameters: Map[String, String] = Map.empty[String, String])(sqlContext: SQLContext) = { - this( - inputRDD, - maybeDataSchema, - maybePartitionSpec, - userDefinedPartitionColumns, - None, - paths, - parameters)(sqlContext) - } - val options: JSONOptions = JSONOptions.createFromConfigMap(parameters) /** Constraints to be imposed on schema to be stored. */ From 1b2c2162af4d5d2d950af94571e69273b49bf913 Mon Sep 17 00:00:00 2001 From: Jacek Laskowski Date: Thu, 7 Jan 2016 21:12:57 +0000 Subject: [PATCH 027/110] =?UTF-8?q?[STREAMING][MINOR]=20More=20contextual?= =?UTF-8?q?=20information=20in=20logs=20+=20minor=20code=20i=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …mprovements Please review and merge at your convenience. Thanks! Author: Jacek Laskowski Closes #10595 from jaceklaskowski/streaming-minor-fixes. --- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 2 +- .../storage/ShuffleBlockFetcherIterator.scala | 4 +- .../spark/network/client/StreamCallback.java | 4 +- .../spark/network/client/TransportClient.java | 2 +- .../spark/network/server/RpcHandler.java | 2 +- .../spark/streaming/StreamingContext.scala | 12 +-- .../spark/streaming/dstream/DStream.scala | 86 +++++++++---------- .../streaming/dstream/InputDStream.scala | 3 +- .../dstream/ReceiverInputDStream.scala | 4 +- .../receiver/ReceivedBlockHandler.scala | 2 +- .../spark/streaming/receiver/Receiver.scala | 4 +- .../receiver/ReceiverSupervisor.scala | 8 +- .../spark/streaming/scheduler/JobSet.scala | 8 +- 14 files changed, 69 insertions(+), 74 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 92438ba892cc0..6b01a10fc136e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -747,7 +747,7 @@ class DAGScheduler( } /** - * Check for waiting or failed stages which are now eligible for resubmission. + * Check for waiting stages which are now eligible for resubmission. * Ordinarily run on every iteration of the event loop. */ private def submitWaitingStages() { 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 5c80ac17b8d90..4479e6875a731 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -59,7 +59,7 @@ private[spark] class BlockResult( * 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. + * Note that [[initialize()]] must be called before the BlockManager is usable. */ private[spark] class BlockManager( executorId: String, 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 0d0448feb5b06..037bec1d9c33b 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -36,7 +36,7 @@ import org.apache.spark.util.Utils * This creates an iterator of (BlockID, InputStream) 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 + * The implementation throttles the remote fetches so they don't exceed maxBytesInFlight to avoid * using too much memory. * * @param context [[TaskContext]], used for metrics update @@ -329,7 +329,7 @@ final class ShuffleBlockFetcherIterator( } /** - * Helper class that ensures a ManagedBuffer is release upon InputStream.close() + * Helper class that ensures a ManagedBuffer is released upon InputStream.close() */ private class BufferReleasingInputStream( private val delegate: InputStream, diff --git a/network/common/src/main/java/org/apache/spark/network/client/StreamCallback.java b/network/common/src/main/java/org/apache/spark/network/client/StreamCallback.java index 51d34cac6e636..29e6a30dc1f67 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/StreamCallback.java +++ b/network/common/src/main/java/org/apache/spark/network/client/StreamCallback.java @@ -21,8 +21,8 @@ import java.nio.ByteBuffer; /** - * Callback for streaming data. Stream data will be offered to the {@link onData(String, ByteBuffer)} - * method as it arrives. Once all the stream data is received, {@link onComplete(String)} will be + * Callback for streaming data. Stream data will be offered to the {@link #onData(String, ByteBuffer)} + * method as it arrives. Once all the stream data is received, {@link #onComplete(String)} will be * called. *

* The network library guarantees that a single thread will call these methods at a time, but 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 c49ca4d5ee925..e15f096d36913 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 @@ -288,7 +288,7 @@ public void send(ByteBuffer message) { /** * Removes any state associated with the given RPC. * - * @param requestId The RPC id returned by {@link #sendRpc(byte[], RpcResponseCallback)}. + * @param requestId The RPC id returned by {@link #sendRpc(ByteBuffer, RpcResponseCallback)}. */ public void removeRpcRequest(long requestId) { handler.removeRpcRequest(requestId); 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 c6ed0f459ad71..a99c3015b0e05 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 @@ -57,7 +57,7 @@ public abstract void receive( /** * Receives an RPC message that does not expect a reply. The default implementation will - * call "{@link receive(TransportClient, byte[], RpcResponseCallback)}" and log a warning if + * call "{@link #receive(TransportClient, ByteBuffer, RpcResponseCallback)}" and log a warning if * any of the callback methods are called. * * @param client A channel client which enables the handler to make requests back to the sender 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 a5ab66697589b..ca0a21fbb79ff 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -226,7 +226,7 @@ class StreamingContext private[streaming] ( * Set the context to periodically checkpoint the DStream operations for driver * fault-tolerance. * @param directory HDFS-compatible directory where the checkpoint data will be reliably stored. - * Note that this must be a fault-tolerant file system like HDFS for + * Note that this must be a fault-tolerant file system like HDFS. */ def checkpoint(directory: String) { if (directory != null) { @@ -274,7 +274,7 @@ class StreamingContext private[streaming] ( * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html * @param receiver Custom implementation of Receiver * - * @deprecated As of 1.0.0", replaced by `receiverStream`. + * @deprecated As of 1.0.0 replaced by `receiverStream`. */ @deprecated("Use receiverStream", "1.0.0") def networkStream[T: ClassTag](receiver: Receiver[T]): ReceiverInputDStream[T] = { @@ -285,7 +285,7 @@ class StreamingContext private[streaming] ( /** * Create an input stream with any arbitrary user implemented receiver. - * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html + * Find more details at http://spark.apache.org/docs/latest/streaming-custom-receivers.html * @param receiver Custom implementation of Receiver */ def receiverStream[T: ClassTag](receiver: Receiver[T]): ReceiverInputDStream[T] = { @@ -549,7 +549,7 @@ class StreamingContext private[streaming] ( // Verify whether the DStream checkpoint is serializable if (isCheckpointingEnabled) { - val checkpoint = new Checkpoint(this, Time.apply(0)) + val checkpoint = new Checkpoint(this, Time(0)) try { Checkpoint.serialize(checkpoint, conf) } catch { @@ -575,9 +575,9 @@ class StreamingContext private[streaming] ( * * Return the current state of the context. The context can be in three possible states - * - * - StreamingContextState.INTIALIZED - The context has been created, but not been started yet. + * - StreamingContextState.INITIALIZED - The context has been created, but not started yet. * Input DStreams, transformations and output operations can be created on the context. - * - StreamingContextState.ACTIVE - The context has been started, and been not stopped. + * - StreamingContextState.ACTIVE - The context has been started, and not stopped. * Input DStreams, transformations and output operations cannot be created on the context. * - StreamingContextState.STOPPED - The context has been stopped and cannot be used any more. */ 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 c59348a89d34f..1dfb4e7abc0ed 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 @@ -103,7 +103,7 @@ abstract class DStream[T: ClassTag] ( // Reference to whole DStream graph private[streaming] var graph: DStreamGraph = null - private[streaming] def isInitialized = (zeroTime != null) + private[streaming] def isInitialized = zeroTime != null // Duration for which the DStream requires its parent DStream to remember each RDD created private[streaming] def parentRememberDuration = rememberDuration @@ -189,15 +189,15 @@ abstract class DStream[T: ClassTag] ( */ private[streaming] def initialize(time: Time) { if (zeroTime != null && zeroTime != time) { - throw new SparkException("ZeroTime is already initialized to " + zeroTime - + ", cannot initialize it again to " + time) + throw new SparkException(s"ZeroTime is already initialized to $zeroTime" + + s", cannot initialize it again to $time") } zeroTime = time // Set the checkpoint interval to be slideDuration or 10 seconds, which ever is larger if (mustCheckpoint && checkpointDuration == null) { checkpointDuration = slideDuration * math.ceil(Seconds(10) / slideDuration).toInt - logInfo("Checkpoint interval automatically set to " + checkpointDuration) + logInfo(s"Checkpoint interval automatically set to $checkpointDuration") } // Set the minimum value of the rememberDuration if not already set @@ -234,7 +234,7 @@ abstract class DStream[T: ClassTag] ( require( !mustCheckpoint || checkpointDuration != null, - "The checkpoint interval for " + this.getClass.getSimpleName + " has not been set." + + s"The checkpoint interval for ${this.getClass.getSimpleName} has not been set." + " Please use DStream.checkpoint() to set the interval." ) @@ -245,53 +245,53 @@ abstract class DStream[T: ClassTag] ( require( checkpointDuration == null || checkpointDuration >= slideDuration, - "The checkpoint interval for " + this.getClass.getSimpleName + " has been set to " + - checkpointDuration + " which is lower than its slide time (" + slideDuration + "). " + - "Please set it to at least " + slideDuration + "." + s"The checkpoint interval for ${this.getClass.getSimpleName} has been set to " + + s"$checkpointDuration which is lower than its slide time ($slideDuration). " + + s"Please set it to at least $slideDuration." ) require( checkpointDuration == null || checkpointDuration.isMultipleOf(slideDuration), - "The checkpoint interval for " + this.getClass.getSimpleName + " has been set to " + - checkpointDuration + " which not a multiple of its slide time (" + slideDuration + "). " + - "Please set it to a multiple of " + slideDuration + "." + s"The checkpoint interval for ${this.getClass.getSimpleName} has been set to " + + s" $checkpointDuration which not a multiple of its slide time ($slideDuration). " + + s"Please set it to a multiple of $slideDuration." ) require( checkpointDuration == null || storageLevel != StorageLevel.NONE, - "" + this.getClass.getSimpleName + " has been marked for checkpointing but the storage " + + s"${this.getClass.getSimpleName} has been marked for checkpointing but the storage " + "level has not been set to enable persisting. Please use DStream.persist() to set the " + "storage level to use memory for better checkpointing performance." ) require( checkpointDuration == null || rememberDuration > checkpointDuration, - "The remember duration for " + this.getClass.getSimpleName + " has been set to " + - rememberDuration + " which is not more than the checkpoint interval (" + - checkpointDuration + "). Please set it to higher than " + checkpointDuration + "." + s"The remember duration for ${this.getClass.getSimpleName} has been set to " + + s" $rememberDuration which is not more than the checkpoint interval" + + s" ($checkpointDuration). Please set it to higher than $checkpointDuration." ) dependencies.foreach(_.validateAtStart()) - logInfo("Slide time = " + slideDuration) - logInfo("Storage level = " + storageLevel) - logInfo("Checkpoint interval = " + checkpointDuration) - logInfo("Remember duration = " + rememberDuration) - logInfo("Initialized and validated " + this) + logInfo(s"Slide time = $slideDuration") + logInfo(s"Storage level = ${storageLevel.description}") + logInfo(s"Checkpoint interval = $checkpointDuration") + logInfo(s"Remember duration = $rememberDuration") + logInfo(s"Initialized and validated $this") } private[streaming] def setContext(s: StreamingContext) { if (ssc != null && ssc != s) { - throw new SparkException("Context is already set in " + this + ", cannot set it again") + throw new SparkException(s"Context must not be set again for $this") } ssc = s - logInfo("Set context for " + this) + logInfo(s"Set context for $this") dependencies.foreach(_.setContext(ssc)) } private[streaming] def setGraph(g: DStreamGraph) { if (graph != null && graph != g) { - throw new SparkException("Graph is already set in " + this + ", cannot set it again") + throw new SparkException(s"Graph must not be set again for $this") } graph = g dependencies.foreach(_.setGraph(graph)) @@ -300,7 +300,7 @@ abstract class DStream[T: ClassTag] ( private[streaming] def remember(duration: Duration) { if (duration != null && (rememberDuration == null || duration > rememberDuration)) { rememberDuration = duration - logInfo("Duration for remembering RDDs set to " + rememberDuration + " for " + this) + logInfo(s"Duration for remembering RDDs set to $rememberDuration for $this") } dependencies.foreach(_.remember(parentRememberDuration)) } @@ -310,11 +310,11 @@ abstract class DStream[T: ClassTag] ( if (!isInitialized) { throw new SparkException (this + " has not been initialized") } else if (time <= zeroTime || ! (time - zeroTime).isMultipleOf(slideDuration)) { - logInfo("Time " + time + " is invalid as zeroTime is " + zeroTime + - " and slideDuration is " + slideDuration + " and difference is " + (time - zeroTime)) + logInfo(s"Time $time is invalid as zeroTime is $zeroTime" + + s" , slideDuration is $slideDuration and difference is ${time - zeroTime}") false } else { - logDebug("Time " + time + " is valid") + logDebug(s"Time $time is valid") true } } @@ -452,20 +452,20 @@ abstract class DStream[T: ClassTag] ( oldRDDs.map(x => s"${x._1} -> ${x._2.id}").mkString(", ") + "]") generatedRDDs --= oldRDDs.keys if (unpersistData) { - logDebug("Unpersisting old RDDs: " + oldRDDs.values.map(_.id).mkString(", ")) + logDebug(s"Unpersisting old RDDs: ${oldRDDs.values.map(_.id).mkString(", ")}") oldRDDs.values.foreach { rdd => rdd.unpersist(false) // Explicitly remove blocks of BlockRDD rdd match { case b: BlockRDD[_] => - logInfo("Removing blocks of RDD " + b + " of time " + time) + logInfo(s"Removing blocks of RDD $b of time $time") b.removeBlocks() case _ => } } } - logDebug("Cleared " + oldRDDs.size + " RDDs that were older than " + - (time - rememberDuration) + ": " + oldRDDs.keys.mkString(", ")) + logDebug(s"Cleared ${oldRDDs.size} RDDs that were older than " + + s"${time - rememberDuration}: ${oldRDDs.keys.mkString(", ")}") dependencies.foreach(_.clearMetadata(time)) } @@ -477,10 +477,10 @@ abstract class DStream[T: ClassTag] ( * this method to save custom checkpoint data. */ private[streaming] def updateCheckpointData(currentTime: Time) { - logDebug("Updating checkpoint data for time " + currentTime) + logDebug(s"Updating checkpoint data for time $currentTime") checkpointData.update(currentTime) dependencies.foreach(_.updateCheckpointData(currentTime)) - logDebug("Updated checkpoint data for time " + currentTime + ": " + checkpointData) + logDebug(s"Updated checkpoint data for time $currentTime: $checkpointData") } private[streaming] def clearCheckpointData(time: Time) { @@ -509,13 +509,13 @@ abstract class DStream[T: ClassTag] ( @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { - logDebug(this.getClass().getSimpleName + ".writeObject used") + logDebug(s"${this.getClass().getSimpleName}.writeObject used") if (graph != null) { graph.synchronized { if (graph.checkpointInProgress) { oos.defaultWriteObject() } else { - val msg = "Object of " + this.getClass.getName + " is being serialized " + + val msg = s"Object of ${this.getClass.getName} is being serialized " + " possibly as a part of closure of an RDD operation. This is because " + " the DStream object is being referred to from within the closure. " + " Please rewrite the RDD operation inside this DStream to avoid this. " + @@ -532,7 +532,7 @@ abstract class DStream[T: ClassTag] ( @throws(classOf[IOException]) private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { - logDebug(this.getClass().getSimpleName + ".readObject used") + logDebug(s"${this.getClass().getSimpleName}.readObject used") ois.defaultReadObject() generatedRDDs = new HashMap[Time, RDD[T]] () } @@ -756,7 +756,7 @@ abstract class DStream[T: ClassTag] ( val firstNum = rdd.take(num + 1) // scalastyle:off println println("-------------------------------------------") - println("Time: " + time) + println(s"Time: $time") println("-------------------------------------------") firstNum.take(num).foreach(println) if (firstNum.length > num) println("...") @@ -903,21 +903,19 @@ abstract class DStream[T: ClassTag] ( val alignedToTime = if ((toTime - zeroTime).isMultipleOf(slideDuration)) { toTime } else { - logWarning("toTime (" + toTime + ") is not a multiple of slideDuration (" - + slideDuration + ")") - toTime.floor(slideDuration, zeroTime) + logWarning(s"toTime ($toTime) is not a multiple of slideDuration ($slideDuration)") + toTime.floor(slideDuration, zeroTime) } val alignedFromTime = if ((fromTime - zeroTime).isMultipleOf(slideDuration)) { fromTime } else { - logWarning("fromTime (" + fromTime + ") is not a multiple of slideDuration (" - + slideDuration + ")") + logWarning(s"fromTime ($fromTime) is not a multiple of slideDuration ($slideDuration)") fromTime.floor(slideDuration, zeroTime) } - logInfo("Slicing from " + fromTime + " to " + toTime + - " (aligned to " + alignedFromTime + " and " + alignedToTime + ")") + logInfo(s"Slicing from $fromTime to $toTime" + + s" (aligned to $alignedFromTime and $alignedToTime)") alignedFromTime.to(alignedToTime, slideDuration).flatMap(time => { if (time >= zeroTime) getOrCompute(time) else None diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala index 95994c983c0cc..d60f418e5c4de 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala @@ -28,7 +28,8 @@ import org.apache.spark.util.Utils /** * This is the abstract base class for all input streams. This class provides methods - * start() and stop() which is called by Spark Streaming system to start and stop receiving data. + * start() and stop() which are called by Spark Streaming system to start and stop + * receiving data, respectively. * Input streams that can generate RDDs from new data by running a service/thread only on * the driver node (that is, without running a receiver on worker nodes), can be * implemented by directly inheriting this InputDStream. For example, 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 a18551fac719a..565b137228d00 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 @@ -32,7 +32,7 @@ import org.apache.spark.streaming.util.WriteAheadLogUtils * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] * that has to start a receiver on worker nodes to receive external data. * Specific implementations of ReceiverInputDStream must - * define `the getReceiver()` function that gets the receiver object of type + * define [[getReceiver]] function that gets the receiver object of type * [[org.apache.spark.streaming.receiver.Receiver]] that will be sent * to the workers to receive data. * @param ssc_ Streaming context that will execute this input stream @@ -121,7 +121,7 @@ abstract class ReceiverInputDStream[T: ClassTag](ssc_ : StreamingContext) } if (validBlockIds.size != blockIds.size) { logWarning("Some blocks could not be recovered as they were not found in memory. " + - "To prevent such data loss, enabled Write Ahead Log (see programming guide " + + "To prevent such data loss, enable Write Ahead Log (see programming guide " + "for more details.") } new BlockRDD[T](ssc.sc, validBlockIds) 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 index 43c605af73716..faa5aca1d8f7a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -69,7 +69,7 @@ private[streaming] class BlockManagerBasedBlockHandler( def storeBlock(blockId: StreamBlockId, block: ReceivedBlock): ReceivedBlockStoreResult = { - var numRecords = None: Option[Long] + var numRecords: Option[Long] = None val putResult: Seq[(BlockId, BlockStatus)] = block match { case ArrayBufferBlock(arrayBuffer) => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala index b08152485ab5b..639f4259e2e73 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -103,7 +103,7 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable /** * This method is called by the system when the receiver is stopped. All resources - * (threads, buffers, etc.) setup in `onStart()` must be cleaned up in this method. + * (threads, buffers, etc.) set up in `onStart()` must be cleaned up in this method. */ def onStop() @@ -273,7 +273,7 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable /** Get the attached supervisor. */ private[streaming] def supervisor: ReceiverSupervisor = { assert(_supervisor != null, - "A ReceiverSupervisor have not been attached to the receiver yet. Maybe you are starting " + + "A ReceiverSupervisor has not been attached to the receiver yet. Maybe you are starting " + "some computation in the receiver before the Receiver.onStart() has been called.") _supervisor } 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 c42a9ac233f87..d0195fb14f0a3 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 @@ -143,10 +143,10 @@ private[streaming] abstract class ReceiverSupervisor( def startReceiver(): Unit = synchronized { try { if (onReceiverStart()) { - logInfo("Starting receiver") + logInfo(s"Starting receiver $streamId") receiverState = Started receiver.onStart() - logInfo("Called receiver onStart") + logInfo(s"Called receiver $streamId onStart") } else { // The driver refused us stop("Registered unsuccessfully because Driver refused to start receiver " + streamId, None) @@ -218,11 +218,9 @@ private[streaming] abstract class ReceiverSupervisor( stopLatch.await() if (stoppingError != null) { logError("Stopped receiver with error: " + stoppingError) + throw stoppingError } else { logInfo("Stopped receiver without error") } - if (stoppingError != null) { - throw stoppingError - } } } 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 f76300351e3c0..6e7232a2a0886 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 @@ -59,17 +59,15 @@ case class JobSet( // Time taken to process all the jobs from the time they were submitted // (i.e. including the time they wait in the streaming scheduler queue) - def totalDelay: Long = { - processingEndTime - time.milliseconds - } + def totalDelay: Long = processingEndTime - time.milliseconds def toBatchInfo: BatchInfo = { BatchInfo( time, streamIdToInputInfo, submissionTime, - if (processingStartTime >= 0) Some(processingStartTime) else None, - if (processingEndTime >= 0) Some(processingEndTime) else None, + if (hasStarted) Some(processingStartTime) else None, + if (hasCompleted) Some(processingEndTime) else None, jobs.map { job => (job.outputOpId, job.toOutputOperationInfo) }.toMap ) } From 8346518357f4a3565ae41e9a5ccd7e2c3ed6c468 Mon Sep 17 00:00:00 2001 From: Darek Blasiak Date: Thu, 7 Jan 2016 21:15:40 +0000 Subject: [PATCH 028/110] [SPARK-12598][CORE] bug in setMinPartitions There is a bug in the calculation of ```maxSplitSize```. The ```totalLen``` should be divided by ```minPartitions``` and not by ```files.size```. Author: Darek Blasiak Closes #10546 from datafarmer/setminpartitionsbug. --- .../scala/org/apache/spark/input/PortableDataStream.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala index 8009491a1b0e0..18cb7631b3d4c 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -41,9 +41,8 @@ private[spark] abstract class StreamFileInputFormat[T] * which is set through setMaxSplitSize */ def setMinPartitions(context: JobContext, minPartitions: Int) { - val files = listStatus(context).asScala - val totalLen = files.map(file => if (file.isDirectory) 0L else file.getLen).sum - val maxSplitSize = Math.ceil(totalLen * 1.0 / files.size).toLong + val totalLen = listStatus(context).asScala.filterNot(_.isDirectory).map(_.getLen).sum + val maxSplitSize = math.ceil(totalLen / math.max(minPartitions, 1.0)).toLong super.setMaxSplitSize(maxSplitSize) } From 34dbc8af21da63702bc0694d471fbfee4cd08dda Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 7 Jan 2016 13:56:34 -0800 Subject: [PATCH 029/110] [SPARK-12580][SQL] Remove string concatenations from usage and extended in @ExpressionDescription Use multi-line string literals for ExpressionDescription with ``// scalastyle:off line.size.limit`` and ``// scalastyle:on line.size.limit`` The policy is here, as describe at https://github.com/apache/spark/pull/10488 Let's use multi-line string literals. If we have to have a line with more than 100 characters, let's use ``// scalastyle:off line.size.limit`` and ``// scalastyle:on line.size.limit`` to just bypass the line number requirement. Author: Kazuaki Ishizaki Closes #10524 from kiszk/SPARK-12580. --- .../spark/sql/catalyst/expressions/misc.scala | 12 +++--- .../expressions/windowExpressions.scala | 38 +++++++++---------- 2 files changed, 25 insertions(+), 25 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index 6697d463614d5..fd95b124b2455 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -58,13 +58,13 @@ case class Md5(child: Expression) extends UnaryExpression with ImplicitCastInput * asking for an unsupported SHA function, the return value is NULL. If either argument is NULL or * the hash length is not one of the permitted values, the return value is NULL. */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = - """_FUNC_(input, bitLength) - Returns a checksum of SHA-2 family as a hex string of the input. - SHA-224, SHA-256, SHA-384, and SHA-512 are supported. Bit length of 0 is equivalent to 256.""" - , - extended = "> SELECT _FUNC_('Spark', 0);\n " + - "'529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b'") + usage = """_FUNC_(input, bitLength) - Returns a checksum of SHA-2 family as a hex string of the input. + SHA-224, SHA-256, SHA-384, and SHA-512 are supported. Bit length of 0 is equivalent to 256.""", + extended = """> SELECT _FUNC_('Spark', 0); + '529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b'""") +// scalastyle:on line.size.limit case class Sha2(left: Expression, right: Expression) extends BinaryExpression with Serializable with ImplicitCastInputTypes { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala index 3934e33628bd8..afe122f6a0e85 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -366,8 +366,8 @@ abstract class OffsetWindowFunction * @param default to use when the input value is null or when the offset is larger than the window. */ @ExpressionDescription(usage = - """_FUNC_(input, offset, default) - LEAD returns the value of 'x' at 'offset' rows after the - current row in the window""") + """_FUNC_(input, offset, default) - LEAD returns the value of 'x' at 'offset' rows + after the current row in the window""") case class Lead(input: Expression, offset: Expression, default: Expression) extends OffsetWindowFunction { @@ -393,8 +393,8 @@ case class Lead(input: Expression, offset: Expression, default: Expression) * @param default to use when the input value is null or when the offset is smaller than the window. */ @ExpressionDescription(usage = - """_FUNC_(input, offset, default) - LAG returns the value of 'x' at 'offset' rows before the - current row in the window""") + """_FUNC_(input, offset, default) - LAG returns the value of 'x' at 'offset' rows + before the current row in the window""") case class Lag(input: Expression, offset: Expression, default: Expression) extends OffsetWindowFunction { @@ -446,9 +446,9 @@ object SizeBasedWindowFunction { * This documentation has been based upon similar documentation for the Hive and Presto projects. */ @ExpressionDescription(usage = - """_FUNC_() - The ROW_NUMBER() function assigns a unique, sequential - number to each row, starting with one, according to the ordering of rows within the window - partition.""") + """_FUNC_() - The ROW_NUMBER() function assigns a unique, sequential number to + each row, starting with one, according to the ordering of rows within + the window partition.""") case class RowNumber() extends RowNumberLike { override val evaluateExpression = rowNumber } @@ -462,8 +462,8 @@ case class RowNumber() extends RowNumberLike { * This documentation has been based upon similar documentation for the Hive and Presto projects. */ @ExpressionDescription(usage = - """_FUNC_() - The CUME_DIST() function computes the position of a value relative to a all values - in the partition.""") + """_FUNC_() - The CUME_DIST() function computes the position of a value relative to + a all values in the partition.""") case class CumeDist() extends RowNumberLike with SizeBasedWindowFunction { override def dataType: DataType = DoubleType // The frame for CUME_DIST is Range based instead of Row based, because CUME_DIST must @@ -494,8 +494,8 @@ case class CumeDist() extends RowNumberLike with SizeBasedWindowFunction { * @param buckets number of buckets to divide the rows in. Default value is 1. */ @ExpressionDescription(usage = - """_FUNC_(x) - The NTILE(n) function divides the rows for each window partition into 'n' buckets - ranging from 1 to at most 'n'.""") + """_FUNC_(x) - The NTILE(n) function divides the rows for each window partition + into 'n' buckets ranging from 1 to at most 'n'.""") case class NTile(buckets: Expression) extends RowNumberLike with SizeBasedWindowFunction { def this() = this(Literal(1)) @@ -602,9 +602,9 @@ abstract class RankLike extends AggregateWindowFunction { * Analyser. */ @ExpressionDescription(usage = - """_FUNC_() - RANK() computes the rank of a value in a group of values. The result is one plus - the number of rows preceding or equal to the current row in the ordering of the partition. Tie - values will produce gaps in the sequence.""") + """_FUNC_() - RANK() computes the rank of a value in a group of values. The result + is one plus the number of rows preceding or equal to the current row in the + ordering of the partition. Tie values will produce gaps in the sequence.""") case class Rank(children: Seq[Expression]) extends RankLike { def this() = this(Nil) override def withOrder(order: Seq[Expression]): Rank = Rank(order) @@ -622,9 +622,9 @@ case class Rank(children: Seq[Expression]) extends RankLike { * Analyser. */ @ExpressionDescription(usage = - """_FUNC_() - The DENSE_RANK() function computes the rank of a value in a group of values. The - result is one plus the previously assigned rank value. Unlike Rank, DenseRank will not produce - gaps in the ranking sequence.""") + """_FUNC_() - The DENSE_RANK() function computes the rank of a value in a group of + values. The result is one plus the previously assigned rank value. Unlike Rank, + DenseRank will not produce gaps in the ranking sequence.""") case class DenseRank(children: Seq[Expression]) extends RankLike { def this() = this(Nil) override def withOrder(order: Seq[Expression]): DenseRank = DenseRank(order) @@ -649,8 +649,8 @@ case class DenseRank(children: Seq[Expression]) extends RankLike { * Analyser. */ @ExpressionDescription(usage = - """_FUNC_() - PERCENT_RANK() The PercentRank function computes the percentage ranking of a value - in a group of values.""") + """_FUNC_() - PERCENT_RANK() The PercentRank function computes the percentage + ranking of a value in a group of values.""") case class PercentRank(children: Seq[Expression]) extends RankLike with SizeBasedWindowFunction { def this() = this(Nil) override def withOrder(order: Seq[Expression]): PercentRank = PercentRank(order) From c0c397509bc909b9bf2d5186182f461155b021ab Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 7 Jan 2016 15:26:55 -0800 Subject: [PATCH 030/110] [SPARK-12510][STREAMING] Refactor ActorReceiver to support Java This PR includes the following changes: 1. Rename `ActorReceiver` to `ActorReceiverSupervisor` 2. Remove `ActorHelper` 3. Add a new `ActorReceiver` for Scala and `JavaActorReceiver` for Java 4. Add `JavaActorWordCount` example Author: Shixiong Zhu Closes #10457 from zsxwing/java-actor-stream. --- .../streaming/JavaActorWordCount.java | 137 ++++++++++++++++++ .../examples/streaming/ActorWordCount.scala | 9 +- .../streaming/zeromq/ZeroMQReceiver.scala | 5 +- project/MimaExcludes.scala | 3 + .../spark/streaming/StreamingContext.scala | 4 +- .../streaming/receiver/ActorReceiver.scala | 64 ++++++-- 6 files changed, 202 insertions(+), 20 deletions(-) create mode 100644 examples/src/main/java/org/apache/spark/examples/streaming/JavaActorWordCount.java diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaActorWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaActorWordCount.java new file mode 100644 index 0000000000000..2377207779fec --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaActorWordCount.java @@ -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.examples.streaming; + +import java.util.Arrays; + +import scala.Tuple2; + +import akka.actor.ActorSelection; +import akka.actor.Props; + +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.streaming.Duration; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.apache.spark.streaming.receiver.JavaActorReceiver; + +/** + * A sample actor as receiver, is also simplest. This receiver actor + * goes and subscribe to a typical publisher/feeder actor and receives + * data. + * + * @see [[org.apache.spark.examples.streaming.FeederActor]] + */ +class JavaSampleActorReceiver extends JavaActorReceiver { + + private final String urlOfPublisher; + + public JavaSampleActorReceiver(String urlOfPublisher) { + this.urlOfPublisher = urlOfPublisher; + } + + private ActorSelection remotePublisher; + + @Override + public void preStart() { + remotePublisher = getContext().actorSelection(urlOfPublisher); + remotePublisher.tell(new SubscribeReceiver(getSelf()), getSelf()); + } + + public void onReceive(Object msg) throws Exception { + store((T) msg); + } + + @Override + public void postStop() { + remotePublisher.tell(new UnsubscribeReceiver(getSelf()), getSelf()); + } +} + +/** + * A sample word count program demonstrating the use of plugging in + * Actor as Receiver + * Usage: JavaActorWordCount + * and describe the AkkaSystem that Spark Sample feeder is running on. + * + * To run this example locally, you may run Feeder Actor as + *

+ *     $ bin/run-example org.apache.spark.examples.streaming.FeederActor localhost 9999
+ * 
+ * and then run the example + *
+ *     $ bin/run-example org.apache.spark.examples.streaming.JavaActorWordCount localhost 9999
+ * 
+ */ +public class JavaActorWordCount { + + public static void main(String[] args) { + if (args.length < 2) { + System.err.println("Usage: JavaActorWordCount "); + System.exit(1); + } + + StreamingExamples.setStreamingLogLevels(); + + final String host = args[0]; + final String port = args[1]; + SparkConf sparkConf = new SparkConf().setAppName("JavaActorWordCount"); + // Create the context and set the batch size + JavaStreamingContext jssc = new JavaStreamingContext(sparkConf, new Duration(2000)); + + String feederActorURI = "akka.tcp://test@" + host + ":" + port + "/user/FeederActor"; + + /* + * Following is the use of actorStream to plug in custom actor as receiver + * + * An important point to note: + * Since Actor may exist outside the spark framework, It is thus user's responsibility + * to ensure the type safety, i.e type of data received and InputDstream + * should be same. + * + * For example: Both actorStream and JavaSampleActorReceiver are parameterized + * to same type to ensure type safety. + */ + JavaDStream lines = jssc.actorStream( + Props.create(JavaSampleActorReceiver.class, feederActorURI), "SampleReceiver"); + + // compute wordcount + lines.flatMap(new FlatMapFunction() { + @Override + public Iterable call(String s) { + return Arrays.asList(s.split("\\s+")); + } + }).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(); + + jssc.start(); + jssc.awaitTermination(); + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala index a47fb7b7d7906..88cdc6bc144e5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala @@ -26,8 +26,7 @@ import akka.actor.{actorRef2Scala, Actor, ActorRef, Props} import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions -import org.apache.spark.streaming.receiver.ActorHelper +import org.apache.spark.streaming.receiver.ActorReceiver import org.apache.spark.util.AkkaUtils case class SubscribeReceiver(receiverActor: ActorRef) @@ -80,7 +79,7 @@ class FeederActor extends Actor { * @see [[org.apache.spark.examples.streaming.FeederActor]] */ class SampleActorReceiver[T: ClassTag](urlOfPublisher: String) -extends Actor with ActorHelper { +extends ActorReceiver { lazy private val remotePublisher = context.actorSelection(urlOfPublisher) @@ -127,9 +126,9 @@ object FeederActor { * and describe the AkkaSystem that Spark Sample feeder is running on. * * To run this example locally, you may run Feeder Actor as - * `$ bin/run-example org.apache.spark.examples.streaming.FeederActor 127.0.0.1 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.FeederActor localhost 9999` * and then run the example - * `$ bin/run-example org.apache.spark.examples.streaming.ActorWordCount 127.0.0.1 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.ActorWordCount localhost 9999` */ object ActorWordCount { def main(args: Array[String]) { diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala index 588e6bac7b14a..506ba8782d3d5 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala @@ -19,12 +19,11 @@ package org.apache.spark.streaming.zeromq import scala.reflect.ClassTag -import akka.actor.Actor import akka.util.ByteString import akka.zeromq._ import org.apache.spark.Logging -import org.apache.spark.streaming.receiver.ActorHelper +import org.apache.spark.streaming.receiver.ActorReceiver /** * A receiver to subscribe to ZeroMQ stream. @@ -33,7 +32,7 @@ private[streaming] class ZeroMQReceiver[T: ClassTag]( publisherUrl: String, subscribe: Subscribe, bytesToObjects: Seq[ByteString] => Iterator[T]) - extends Actor with ActorHelper with Logging { + extends ActorReceiver with Logging { override def preStart(): Unit = { ZeroMQExtension(context.system) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 43ca4690dc2bb..69e5bc881b593 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -119,6 +119,9 @@ object MimaExcludes { ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector$Multiplier"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector$") + ) ++ Seq( + // SPARK-12510 Refactor ActorReceiver to support Java + ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver") ) case v if v.startsWith("1.6") => 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 ca0a21fbb79ff..ba509a1030af7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -41,7 +41,7 @@ import org.apache.spark.serializer.SerializationDebugger import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContextState._ import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receiver.{ActorReceiver, ActorSupervisorStrategy, Receiver} +import org.apache.spark.streaming.receiver.{ActorReceiverSupervisor, ActorSupervisorStrategy, Receiver} import org.apache.spark.streaming.scheduler.{JobScheduler, StreamingListener} import org.apache.spark.streaming.ui.{StreamingJobProgressListener, StreamingTab} import org.apache.spark.util.{AsynchronousListenerBus, CallSite, ShutdownHookManager, ThreadUtils, Utils} @@ -312,7 +312,7 @@ class StreamingContext private[streaming] ( storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, supervisorStrategy: SupervisorStrategy = ActorSupervisorStrategy.defaultStrategy ): ReceiverInputDStream[T] = withNamedScope("actor stream") { - receiverStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy)) + receiverStream(new ActorReceiverSupervisor[T](props, name, storageLevel, supervisorStrategy)) } /** 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 7ec74016a1c2c..0eabf3d260b26 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 @@ -47,13 +47,12 @@ object ActorSupervisorStrategy { /** * :: DeveloperApi :: - * A receiver trait to be mixed in with your Actor to gain access to - * the API for pushing received data into Spark Streaming for being processed. + * A base Actor that provides APIs for pushing received data into Spark Streaming for processing. * * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html * * @example {{{ - * class MyActor extends Actor with ActorHelper{ + * class MyActor extends ActorReceiver { * def receive { * case anything: String => store(anything) * } @@ -69,13 +68,60 @@ object ActorSupervisorStrategy { * should be same. */ @DeveloperApi -trait ActorHelper extends Logging{ +abstract class ActorReceiver extends Actor { - 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]) { + context.parent ! IteratorData(iter) + } + + /** + * Store the bytes of received data as a data block into Spark's memory. Note + * that the data in the ByteBuffer must be serialized using the same serializer + * that Spark is configured to use. + */ + def store(bytes: ByteBuffer) { + context.parent ! ByteBufferData(bytes) + } + + /** + * Store a single item of received data to Spark's memory. + * These single items will be aggregated together into data blocks before + * being pushed into Spark's memory. + */ + def store[T](item: T) { + context.parent ! SingleItemData(item) + } +} + +/** + * :: DeveloperApi :: + * A Java UntypedActor that provides APIs for pushing received data into Spark Streaming for + * processing. + * + * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html + * + * @example {{{ + * class MyActor extends JavaActorReceiver { + * def receive { + * case anything: String => store(anything) + * } + * } + * + * // Can be used with an actorStream as follows + * ssc.actorStream[String](Props(new MyActor),"MyActorReceiver") + * + * }}} + * + * @note Since Actor may exist outside the spark framework, It is thus user's responsibility + * to ensure the type safety, i.e parametrized type of push block and InputDStream + * should be same. + */ +@DeveloperApi +abstract class JavaActorReceiver extends UntypedActor { /** Store an iterator of received data as a data block into Spark's memory. */ def store[T](iter: Iterator[T]) { - logDebug("Storing iterator") context.parent ! IteratorData(iter) } @@ -85,7 +131,6 @@ trait ActorHelper extends Logging{ * that Spark is configured to use. */ def store(bytes: ByteBuffer) { - logDebug("Storing Bytes") context.parent ! ByteBufferData(bytes) } @@ -95,7 +140,6 @@ trait ActorHelper extends Logging{ * being pushed into Spark's memory. */ def store[T](item: T) { - logDebug("Storing item") context.parent ! SingleItemData(item) } } @@ -104,7 +148,7 @@ trait ActorHelper extends Logging{ * :: DeveloperApi :: * Statistics for querying the supervisor about state of workers. Used in * conjunction with `StreamingContext.actorStream` and - * [[org.apache.spark.streaming.receiver.ActorHelper]]. + * [[org.apache.spark.streaming.receiver.ActorReceiver]]. */ @DeveloperApi case class Statistics(numberOfMsgs: Int, @@ -137,7 +181,7 @@ private[streaming] case class ByteBufferData(bytes: ByteBuffer) extends ActorRec * context.parent ! Props(new Worker, "Worker") * }}} */ -private[streaming] class ActorReceiver[T: ClassTag]( +private[streaming] class ActorReceiverSupervisor[T: ClassTag]( props: Props, name: String, storageLevel: StorageLevel, From 5a4021998ab0f1c8bbb610eceecdf879d149a7b8 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 7 Jan 2016 17:21:03 -0800 Subject: [PATCH 031/110] [SPARK-12604][CORE] Addendum - use casting vs mapValues for countBy{Key,Value} Per rxin, let's use the casting for countByKey and countByValue as well. Let's see if this passes. Author: Sean Owen Closes #10641 from srowen/SPARK-12604.2. --- core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala | 2 +- core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 2 +- 2 files changed, 2 insertions(+), 2 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 76752e1fde663..59af1052ebd05 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 @@ -296,7 +296,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** Count the number of elements for each key, and return the result to the master as a Map. */ def countByKey(): java.util.Map[K, jl.Long] = - mapAsSerializableJavaMap(rdd.countByKey().mapValues(jl.Long.valueOf)) + mapAsSerializableJavaMap(rdd.countByKey()).asInstanceOf[java.util.Map[K, jl.Long]] /** * Approximate version of countByKey that can return a partial result if it does 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 1b1a9dce397fd..242438237f987 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 @@ -448,7 +448,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, jl.Long] = - mapAsSerializableJavaMap(rdd.countByValue().mapValues(jl.Long.valueOf)) + mapAsSerializableJavaMap(rdd.countByValue()).asInstanceOf[java.util.Map[T, jl.Long]] /** * (Experimental) Approximate version of countByValue(). From c94199e977279d9b4658297e8108b46bdf30157b Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 7 Jan 2016 17:37:46 -0800 Subject: [PATCH 032/110] [SPARK-12507][STREAMING][DOCUMENT] Expose closeFileAfterWrite and allowBatching configurations for Streaming /cc tdas brkyvz Author: Shixiong Zhu Closes #10453 from zsxwing/streaming-conf. --- docs/configuration.md | 18 ++++++++++++++++++ docs/streaming-programming-guide.md | 12 +++++------- 2 files changed, 23 insertions(+), 7 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 6bd0658b3e056..08392c39187b9 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1574,6 +1574,24 @@ Apart from these, the following properties are also available, and may be useful How many batches the Spark Streaming UI and status APIs remember before garbage collecting. + + spark.streaming.driver.writeAheadLog.closeFileAfterWrite + false + + Whether to close the file after writing a write ahead log record on the driver. Set this to 'true' + when you want to use S3 (or any file system that does not support flushing) for the metadata WAL + on the driver. + + + + spark.streaming.receiver.writeAheadLog.closeFileAfterWrite + false + + Whether to close the file after writing a write ahead log record on the receivers. Set this to 'true' + when you want to use S3 (or any file system that does not support flushing) for the data WAL + on the receivers. + + #### SparkR diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 3b071c7da5596..1edc0fe34706b 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1985,7 +1985,11 @@ To run a Spark Streaming applications, you need to have the following. to increase aggregate throughput. Additionally, it is recommended that the replication of the received data within Spark be disabled when the write ahead log is enabled as the log is already stored in a replicated storage system. This can be done by setting the storage level for the - input stream to `StorageLevel.MEMORY_AND_DISK_SER`. + input stream to `StorageLevel.MEMORY_AND_DISK_SER`. While using S3 (or any file system that + does not support flushing) for _write ahead logs_, please remember to enable + `spark.streaming.driver.writeAheadLog.closeFileAfterWrite` and + `spark.streaming.receiver.writeAheadLog.closeFileAfterWrite`. See + [Spark Streaming Configuration](configuration.html#spark-streaming) for more details. - *Setting the max receiving rate* - If the cluster resources is not large enough for the streaming application to process data as fast as it is being received, the receivers can be rate limited @@ -2023,12 +2027,6 @@ contains serialized Scala/Java/Python objects and trying to deserialize objects modified classes may lead to errors. In this case, either start the upgraded app with a different checkpoint directory, or delete the previous checkpoint directory. -### Other Considerations -{:.no_toc} -If the data is being received by the receivers faster than what can be processed, -you can limit the rate by setting the [configuration parameter](configuration.html#spark-streaming) -`spark.streaming.receiver.maxRate`. - *** ## Monitoring Applications From 28e0e500a2062baeda8c887e17dc8ab2b7d7d4b4 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 7 Jan 2016 17:46:24 -0800 Subject: [PATCH 033/110] [SPARK-12591][STREAMING] Register OpenHashMapBasedStateMap for Kryo The default serializer in Kryo is FieldSerializer and it ignores transient fields and never calls `writeObject` or `readObject`. So we should register OpenHashMapBasedStateMap using `DefaultSerializer` to make it work with Kryo. Author: Shixiong Zhu Closes #10609 from zsxwing/SPARK-12591. --- .../spark/serializer/KryoSerializer.scala | 24 ++++- .../serializer/KryoSerializerSuite.scala | 20 +++- project/MimaExcludes.scala | 4 + .../spark/streaming/util/StateMap.scala | 71 ++++++++++---- .../spark/streaming/StateMapSuite.scala | 96 ++++++++++++++++--- 5 files changed, 174 insertions(+), 41 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 bc9fd50c2cd2b..150ddc12e0694 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -17,7 +17,7 @@ package org.apache.spark.serializer -import java.io.{DataInput, DataOutput, EOFException, InputStream, IOException, OutputStream} +import java.io._ import java.nio.ByteBuffer import javax.annotation.Nullable @@ -378,18 +378,24 @@ private[serializer] object KryoSerializer { private val toRegisterSerializer = Map[Class[_], KryoClassSerializer[_]]( classOf[RoaringBitmap] -> new KryoClassSerializer[RoaringBitmap]() { override def write(kryo: Kryo, output: KryoOutput, bitmap: RoaringBitmap): Unit = { - bitmap.serialize(new KryoOutputDataOutputBridge(output)) + bitmap.serialize(new KryoOutputObjectOutputBridge(kryo, output)) } override def read(kryo: Kryo, input: KryoInput, cls: Class[RoaringBitmap]): RoaringBitmap = { val ret = new RoaringBitmap - ret.deserialize(new KryoInputDataInputBridge(input)) + ret.deserialize(new KryoInputObjectInputBridge(kryo, input)) ret } } ) } -private[serializer] class KryoInputDataInputBridge(input: KryoInput) extends DataInput { +/** + * This is a bridge class to wrap KryoInput as an InputStream and ObjectInput. It forwards all + * methods of InputStream and ObjectInput to KryoInput. It's usually helpful when an API expects + * an InputStream or ObjectInput but you want to use Kryo. + */ +private[spark] class KryoInputObjectInputBridge( + kryo: Kryo, input: KryoInput) extends FilterInputStream(input) with ObjectInput { override def readLong(): Long = input.readLong() override def readChar(): Char = input.readChar() override def readFloat(): Float = input.readFloat() @@ -408,9 +414,16 @@ private[serializer] class KryoInputDataInputBridge(input: KryoInput) extends Dat override def readBoolean(): Boolean = input.readBoolean() override def readUnsignedByte(): Int = input.readByteUnsigned() override def readDouble(): Double = input.readDouble() + override def readObject(): AnyRef = kryo.readClassAndObject(input) } -private[serializer] class KryoOutputDataOutputBridge(output: KryoOutput) extends DataOutput { +/** + * This is a bridge class to wrap KryoOutput as an OutputStream and ObjectOutput. It forwards all + * methods of OutputStream and ObjectOutput to KryoOutput. It's usually helpful when an API expects + * an OutputStream or ObjectOutput but you want to use Kryo. + */ +private[spark] class KryoOutputObjectOutputBridge( + kryo: Kryo, output: KryoOutput) extends FilterOutputStream(output) with ObjectOutput { override def writeFloat(v: Float): Unit = output.writeFloat(v) // There is no "readChars" counterpart, except maybe "readLine", which is not supported override def writeChars(s: String): Unit = throw new UnsupportedOperationException("writeChars") @@ -426,6 +439,7 @@ private[serializer] class KryoOutputDataOutputBridge(output: KryoOutput) extends override def writeChar(v: Int): Unit = output.writeChar(v.toChar) override def writeLong(v: Long): Unit = output.writeLong(v) override def writeByte(v: Int): Unit = output.writeByte(v) + override def writeObject(obj: AnyRef): Unit = kryo.writeClassAndObject(output, obj) } /** 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 8f9b453a6eeec..f869bcd708619 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -362,19 +362,35 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { bitmap.add(1) bitmap.add(3) bitmap.add(5) - bitmap.serialize(new KryoOutputDataOutputBridge(output)) + // Ignore Kryo because it doesn't use writeObject + bitmap.serialize(new KryoOutputObjectOutputBridge(null, output)) output.flush() output.close() val inStream = new FileInputStream(tmpfile) val input = new KryoInput(inStream) val ret = new RoaringBitmap - ret.deserialize(new KryoInputDataInputBridge(input)) + // Ignore Kryo because it doesn't use readObject + ret.deserialize(new KryoInputObjectInputBridge(null, input)) input.close() assert(ret == bitmap) Utils.deleteRecursively(dir) } + test("KryoOutputObjectOutputBridge.writeObject and KryoInputObjectInputBridge.readObject") { + val kryo = new KryoSerializer(conf).newKryo() + + val bytesOutput = new ByteArrayOutputStream() + val objectOutput = new KryoOutputObjectOutputBridge(kryo, new KryoOutput(bytesOutput)) + objectOutput.writeObject("test") + objectOutput.close() + + val bytesInput = new ByteArrayInputStream(bytesOutput.toByteArray) + val objectInput = new KryoInputObjectInputBridge(kryo, new KryoInput(bytesInput)) + assert(objectInput.readObject() === "test") + objectInput.close() + } + test("getAutoReset") { val ser = new KryoSerializer(new SparkConf).newInstance().asInstanceOf[KryoSerializerInstance] assert(ser.getAutoReset) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 69e5bc881b593..40559a0910ce8 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -119,6 +119,10 @@ object MimaExcludes { ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector$Multiplier"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector$") + ) ++ Seq( + // SPARK-12591 Register OpenHashMapBasedStateMap for Kryo + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.KryoInputDataInputBridge"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.KryoOutputDataOutputBridge") ) ++ Seq( // SPARK-12510 Refactor ActorReceiver to support Java ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala index 3f139ad138c88..4e5baebaae04b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala @@ -17,16 +17,20 @@ package org.apache.spark.streaming.util -import java.io.{ObjectInputStream, ObjectOutputStream} +import java.io._ import scala.reflect.ClassTag +import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import com.esotericsoftware.kryo.io.{Input, Output} + import org.apache.spark.SparkConf +import org.apache.spark.serializer.{KryoOutputObjectOutputBridge, KryoInputObjectInputBridge} import org.apache.spark.streaming.util.OpenHashMapBasedStateMap._ import org.apache.spark.util.collection.OpenHashMap /** Internal interface for defining the map that keeps track of sessions. */ -private[streaming] abstract class StateMap[K: ClassTag, S: ClassTag] extends Serializable { +private[streaming] abstract class StateMap[K, S] extends Serializable { /** Get the state for a key if it exists */ def get(key: K): Option[S] @@ -54,7 +58,7 @@ private[streaming] abstract class StateMap[K: ClassTag, S: ClassTag] extends Ser /** Companion object for [[StateMap]], with utility methods */ private[streaming] object StateMap { - def empty[K: ClassTag, S: ClassTag]: StateMap[K, S] = new EmptyStateMap[K, S] + def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S] def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = { val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold", @@ -64,7 +68,7 @@ private[streaming] object StateMap { } /** Implementation of StateMap interface representing an empty map */ -private[streaming] class EmptyStateMap[K: ClassTag, S: ClassTag] extends StateMap[K, S] { +private[streaming] class EmptyStateMap[K, S] extends StateMap[K, S] { override def put(key: K, session: S, updateTime: Long): Unit = { throw new NotImplementedError("put() should not be called on an EmptyStateMap") } @@ -77,21 +81,26 @@ private[streaming] class EmptyStateMap[K: ClassTag, S: ClassTag] extends StateMa } /** Implementation of StateMap based on Spark's [[org.apache.spark.util.collection.OpenHashMap]] */ -private[streaming] class OpenHashMapBasedStateMap[K: ClassTag, S: ClassTag]( +private[streaming] class OpenHashMapBasedStateMap[K, S]( @transient @volatile var parentStateMap: StateMap[K, S], - initialCapacity: Int = DEFAULT_INITIAL_CAPACITY, - deltaChainThreshold: Int = DELTA_CHAIN_LENGTH_THRESHOLD - ) extends StateMap[K, S] { self => + private var initialCapacity: Int = DEFAULT_INITIAL_CAPACITY, + private var deltaChainThreshold: Int = DELTA_CHAIN_LENGTH_THRESHOLD + )(implicit private var keyClassTag: ClassTag[K], private var stateClassTag: ClassTag[S]) + extends StateMap[K, S] with KryoSerializable { self => - def this(initialCapacity: Int, deltaChainThreshold: Int) = this( + def this(initialCapacity: Int, deltaChainThreshold: Int) + (implicit keyClassTag: ClassTag[K], stateClassTag: ClassTag[S]) = this( new EmptyStateMap[K, S], initialCapacity = initialCapacity, deltaChainThreshold = deltaChainThreshold) - def this(deltaChainThreshold: Int) = this( + def this(deltaChainThreshold: Int) + (implicit keyClassTag: ClassTag[K], stateClassTag: ClassTag[S]) = this( initialCapacity = DEFAULT_INITIAL_CAPACITY, deltaChainThreshold = deltaChainThreshold) - def this() = this(DELTA_CHAIN_LENGTH_THRESHOLD) + def this()(implicit keyClassTag: ClassTag[K], stateClassTag: ClassTag[S]) = { + this(DELTA_CHAIN_LENGTH_THRESHOLD) + } require(initialCapacity >= 1, "Invalid initial capacity") require(deltaChainThreshold >= 1, "Invalid delta chain threshold") @@ -206,11 +215,7 @@ private[streaming] class OpenHashMapBasedStateMap[K: ClassTag, S: ClassTag]( * Serialize the map data. Besides serialization, this method actually compact the deltas * (if needed) in a single pass over all the data in the map. */ - - private def writeObject(outputStream: ObjectOutputStream): Unit = { - // Write all the non-transient fields, especially class tags, etc. - outputStream.defaultWriteObject() - + private def writeObjectInternal(outputStream: ObjectOutput): Unit = { // Write the data in the delta of this state map outputStream.writeInt(deltaMap.size) val deltaMapIterator = deltaMap.iterator @@ -262,11 +267,7 @@ private[streaming] class OpenHashMapBasedStateMap[K: ClassTag, S: ClassTag]( } /** Deserialize the map data. */ - private def readObject(inputStream: ObjectInputStream): Unit = { - - // Read the non-transient fields, especially class tags, etc. - inputStream.defaultReadObject() - + private def readObjectInternal(inputStream: ObjectInput): Unit = { // Read the data of the delta val deltaMapSize = inputStream.readInt() deltaMap = if (deltaMapSize != 0) { @@ -309,6 +310,34 @@ private[streaming] class OpenHashMapBasedStateMap[K: ClassTag, S: ClassTag]( } parentStateMap = newParentSessionStore } + + private def writeObject(outputStream: ObjectOutputStream): Unit = { + // Write all the non-transient fields, especially class tags, etc. + outputStream.defaultWriteObject() + writeObjectInternal(outputStream) + } + + private def readObject(inputStream: ObjectInputStream): Unit = { + // Read the non-transient fields, especially class tags, etc. + inputStream.defaultReadObject() + readObjectInternal(inputStream) + } + + override def write(kryo: Kryo, output: Output): Unit = { + output.writeInt(initialCapacity) + output.writeInt(deltaChainThreshold) + kryo.writeClassAndObject(output, keyClassTag) + kryo.writeClassAndObject(output, stateClassTag) + writeObjectInternal(new KryoOutputObjectOutputBridge(kryo, output)) + } + + override def read(kryo: Kryo, input: Input): Unit = { + initialCapacity = input.readInt() + deltaChainThreshold = input.readInt() + keyClassTag = kryo.readClassAndObject(input).asInstanceOf[ClassTag[K]] + stateClassTag = kryo.readClassAndObject(input).asInstanceOf[ClassTag[S]] + readObjectInternal(new KryoInputObjectInputBridge(kryo, input)) + } } /** diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala index c4a01eaea739e..ea32bbf95ce59 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala @@ -17,15 +17,23 @@ package org.apache.spark.streaming +import org.apache.spark.streaming.rdd.MapWithStateRDDRecord + import scala.collection.{immutable, mutable, Map} +import scala.reflect.ClassTag import scala.util.Random -import org.apache.spark.SparkFunSuite +import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import com.esotericsoftware.kryo.io.{Output, Input} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.serializer._ import org.apache.spark.streaming.util.{EmptyStateMap, OpenHashMapBasedStateMap, StateMap} -import org.apache.spark.util.Utils class StateMapSuite extends SparkFunSuite { + private val conf = new SparkConf() + test("EmptyStateMap") { val map = new EmptyStateMap[Int, Int] intercept[scala.NotImplementedError] { @@ -128,17 +136,17 @@ class StateMapSuite extends SparkFunSuite { map1.put(2, 200, 2) testSerialization(map1, "error deserializing and serialized map with data + no delta") - val map2 = map1.copy() + val map2 = map1.copy().asInstanceOf[OpenHashMapBasedStateMap[Int, Int]] // Do not test compaction - assert(map2.asInstanceOf[OpenHashMapBasedStateMap[_, _]].shouldCompact === false) + assert(map2.shouldCompact === false) testSerialization(map2, "error deserializing and serialized map with 1 delta + no new data") map2.put(3, 300, 3) map2.put(4, 400, 4) testSerialization(map2, "error deserializing and serialized map with 1 delta + new data") - val map3 = map2.copy() - assert(map3.asInstanceOf[OpenHashMapBasedStateMap[_, _]].shouldCompact === false) + val map3 = map2.copy().asInstanceOf[OpenHashMapBasedStateMap[Int, Int]] + assert(map3.shouldCompact === false) testSerialization(map3, "error deserializing and serialized map with 2 delta + no new data") map3.put(3, 600, 3) map3.remove(2) @@ -267,18 +275,25 @@ class StateMapSuite extends SparkFunSuite { assertMap(stateMap, refMap.toMap, time, "Final state map does not match reference map") } - private def testSerialization[MapType <: StateMap[Int, Int]]( - map: MapType, msg: String): MapType = { - val deserMap = Utils.deserialize[MapType]( - Utils.serialize(map), Thread.currentThread().getContextClassLoader) + private def testSerialization[T: ClassTag]( + map: OpenHashMapBasedStateMap[T, T], msg: String): OpenHashMapBasedStateMap[T, T] = { + testSerialization(new JavaSerializer(conf), map, msg) + testSerialization(new KryoSerializer(conf), map, msg) + } + + private def testSerialization[T : ClassTag]( + serializer: Serializer, + map: OpenHashMapBasedStateMap[T, T], + msg: String): OpenHashMapBasedStateMap[T, T] = { + val deserMap = serializeAndDeserialize(serializer, map) assertMap(deserMap, map, 1, msg) deserMap } // Assert whether all the data and operations on a state map matches that of a reference state map - private def assertMap( - mapToTest: StateMap[Int, Int], - refMapToTestWith: StateMap[Int, Int], + private def assertMap[T]( + mapToTest: StateMap[T, T], + refMapToTestWith: StateMap[T, T], time: Long, msg: String): Unit = { withClue(msg) { @@ -321,4 +336,59 @@ class StateMapSuite extends SparkFunSuite { } } } + + test("OpenHashMapBasedStateMap - serializing and deserializing with KryoSerializable states") { + val map = new OpenHashMapBasedStateMap[KryoState, KryoState]() + map.put(new KryoState("a"), new KryoState("b"), 1) + testSerialization( + new KryoSerializer(conf), map, "error deserializing and serialized KryoSerializable states") + } + + test("EmptyStateMap - serializing and deserializing") { + val map = StateMap.empty[KryoState, KryoState] + // Since EmptyStateMap doesn't contains any date, KryoState won't break JavaSerializer. + assert(serializeAndDeserialize(new JavaSerializer(conf), map). + isInstanceOf[EmptyStateMap[KryoState, KryoState]]) + assert(serializeAndDeserialize(new KryoSerializer(conf), map). + isInstanceOf[EmptyStateMap[KryoState, KryoState]]) + } + + test("MapWithStateRDDRecord - serializing and deserializing with KryoSerializable states") { + val map = new OpenHashMapBasedStateMap[KryoState, KryoState]() + map.put(new KryoState("a"), new KryoState("b"), 1) + + val record = + MapWithStateRDDRecord[KryoState, KryoState, KryoState](map, Seq(new KryoState("c"))) + val deserRecord = serializeAndDeserialize(new KryoSerializer(conf), record) + assert(!(record eq deserRecord)) + assert(record.stateMap.getAll().toSeq === deserRecord.stateMap.getAll().toSeq) + assert(record.mappedData === deserRecord.mappedData) + } + + private def serializeAndDeserialize[T: ClassTag](serializer: Serializer, t: T): T = { + val serializerInstance = serializer.newInstance() + serializerInstance.deserialize[T]( + serializerInstance.serialize(t), Thread.currentThread().getContextClassLoader) + } +} + +/** A class that only supports Kryo serialization. */ +private[streaming] final class KryoState(var state: String) extends KryoSerializable { + + override def write(kryo: Kryo, output: Output): Unit = { + kryo.writeClassAndObject(output, state) + } + + override def read(kryo: Kryo, input: Input): Unit = { + state = kryo.readClassAndObject(input).asInstanceOf[String] + } + + override def equals(other: Any): Boolean = other match { + case that: KryoState => state == that.state + case _ => false + } + + override def hashCode(): Int = { + if (state == null) 0 else state.hashCode() + } } From 5028a001d51a9e9a13e3c39f6a080618f3425d87 Mon Sep 17 00:00:00 2001 From: Kevin Yu Date: Thu, 7 Jan 2016 21:13:17 -0800 Subject: [PATCH 034/110] [SPARK-12317][SQL] Support units (m,k,g) in SQLConf This PR is continue from previous closed PR 10314. In this PR, SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE will be taken memory string conventions as input. For example, the user can now specify 10g for SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE in SQLConf file. marmbrus srowen : Can you help review this code changes ? Thanks. Author: Kevin Yu Closes #10629 from kevinyu98/spark-12317. --- .../scala/org/apache/spark/sql/SQLConf.scala | 22 ++++++++++- .../org/apache/spark/sql/SQLConfSuite.scala | 39 +++++++++++++++++++ 2 files changed, 60 insertions(+), 1 deletion(-) 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 26c00dc250b4b..7976795ff5919 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 org.apache.parquet.hadoop.ParquetOutputCommitter import org.apache.spark.sql.catalyst.CatalystConf import org.apache.spark.sql.catalyst.parser.ParserConf +import org.apache.spark.util.Utils //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines the configuration options for Spark SQL. @@ -115,6 +116,25 @@ private[spark] object SQLConf { } }, _.toString, doc, isPublic) + def longMemConf( + key: String, + defaultValue: Option[Long] = None, + doc: String = "", + isPublic: Boolean = true): SQLConfEntry[Long] = + SQLConfEntry(key, defaultValue, { v => + try { + v.toLong + } catch { + case _: NumberFormatException => + try { + Utils.byteStringAsBytes(v) + } catch { + case _: NumberFormatException => + throw new IllegalArgumentException(s"$key should be long, but was $v") + } + } + }, _.toString, doc, isPublic) + def doubleConf( key: String, defaultValue: Option[Double] = None, @@ -235,7 +255,7 @@ private[spark] object SQLConf { doc = "The default number of partitions to use when shuffling data for joins or aggregations.") val SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE = - longConf("spark.sql.adaptive.shuffle.targetPostShuffleInputSize", + longMemConf("spark.sql.adaptive.shuffle.targetPostShuffleInputSize", defaultValue = Some(64 * 1024 * 1024), doc = "The target post-shuffle input size in bytes of a task.") 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 43300cd635c05..a2eddc8fe173e 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 @@ -92,4 +92,43 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } assert(e.getMessage === s"${SQLConf.CASE_SENSITIVE.key} should be boolean, but was 10") } + + test("Test SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE's method") { + sqlContext.conf.clear() + + sqlContext.setConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "100") + assert(sqlContext.conf.targetPostShuffleInputSize === 100) + + sqlContext.setConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "1k") + assert(sqlContext.conf.targetPostShuffleInputSize === 1024) + + sqlContext.setConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "1M") + assert(sqlContext.conf.targetPostShuffleInputSize === 1048576) + + sqlContext.setConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "1g") + assert(sqlContext.conf.targetPostShuffleInputSize === 1073741824) + + sqlContext.setConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "-1") + assert(sqlContext.conf.targetPostShuffleInputSize === -1) + + // Test overflow exception + intercept[IllegalArgumentException] { + // This value exceeds Long.MaxValue + // Utils.byteStringAsBytes("90000000000g") + sqlContext.setConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "90000000000g") + } + + intercept[IllegalArgumentException] { + // This value less than Int.MinValue + // Utils.byteStringAsBytes("-90000000000g") + sqlContext.setConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "-90000000000g") + } + // Test invalid input + intercept[IllegalArgumentException] { + // This value exceeds Long.MaxValue + // Utils.byteStringAsBytes("-1g") + sqlContext.setConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "-1g") + } + sqlContext.conf.clear() + } } From 726bd3c4ece33667096f04be4d3e1ea13048a1af Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 7 Jan 2016 21:15:43 -0800 Subject: [PATCH 035/110] Fix indentation for the previous patch. --- .../org/apache/spark/sql/SQLConfSuite.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) 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 a2eddc8fe173e..cf0701eca29ea 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 @@ -113,22 +113,20 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { // Test overflow exception intercept[IllegalArgumentException] { - // This value exceeds Long.MaxValue - // Utils.byteStringAsBytes("90000000000g") - sqlContext.setConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "90000000000g") + // This value exceeds Long.MaxValue + sqlContext.setConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "90000000000g") } intercept[IllegalArgumentException] { - // This value less than Int.MinValue - // Utils.byteStringAsBytes("-90000000000g") + // This value less than Int.MinValue sqlContext.setConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "-90000000000g") - } + } + // Test invalid input intercept[IllegalArgumentException] { - // This value exceeds Long.MaxValue - // Utils.byteStringAsBytes("-1g") - sqlContext.setConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "-1g") - } + // This value exceeds Long.MaxValue + sqlContext.setConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "-1g") + } sqlContext.conf.clear() } } From 794ea553bd0fcfece15b610b47ee86d6644134c9 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 8 Jan 2016 00:53:15 -0800 Subject: [PATCH 036/110] [SPARK-12692][BUILD] Scala style: check no white space before comma and colon We should not put a white space before `,` and `:` so let's check it. Because there are lots of style violations, first, I'd like to add a checker, enable and let the level `warning`. Then, I'd like to fix the style step by step. Author: Kousuke Saruta Closes #10643 from sarutak/SPARK-12692. --- scalastyle-config.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/scalastyle-config.xml b/scalastyle-config.xml index ee855ca0e09cb..9714c46fe99a0 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -218,6 +218,12 @@ This file is divided into 3 sections: + + + + COLON, COMMA + + From b9c835337880f57fe8b953962913bcc524162348 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 8 Jan 2016 17:47:44 +0000 Subject: [PATCH 037/110] [SPARK-12618][CORE][STREAMING][SQL] Clean up build warnings: 2.0.0 edition Fix most build warnings: mostly deprecated API usages. I'll annotate some of the changes below. CC rxin who is leading the charge to remove the deprecated APIs. Author: Sean Owen Closes #10570 from srowen/SPARK-12618. --- .../test/scala/org/apache/spark/Smuggle.scala | 1 + ...avaBinaryClassificationMetricsExample.java | 5 +- .../mllib/JavaRankingMetricsExample.java | 21 ++++-- .../JavaRecoverableNetworkWordCount.java | 8 +-- .../streaming/JavaSqlNetworkWordCount.java | 8 +-- .../JavaTwitterHashTagJoinSentiments.java | 36 +++++------ .../apache/spark/examples/SparkHdfsLR.scala | 2 +- .../spark/examples/SparkTachyonHdfsLR.scala | 2 +- .../kafka/JavaDirectKafkaStreamSuite.java | 7 +- .../streaming/kafka/JavaKafkaStreamSuite.java | 8 +-- .../kinesis/KinesisStreamSuite.scala | 8 +-- .../spark/mllib/clustering/KMeans.scala | 8 +-- .../mllib/recommendation/JavaALSSuite.java | 4 +- .../JavaIsotonicRegressionSuite.java | 18 +++--- python/pyspark/mllib/clustering.py | 2 +- .../expressions/ExpressionEvalHelper.scala | 8 +-- .../catalyst/util/DateTimeUtilsSuite.scala | 3 - .../SpecificParquetRecordReaderBase.java | 19 +++--- .../spark/sql/ColumnExpressionSuite.scala | 4 +- .../org/apache/spark/sql/QueryTest.scala | 5 +- .../columnar/ColumnarTestUtils.scala | 1 + .../apache/spark/streaming/JavaAPISuite.java | 4 +- .../streaming/JavaMapWithStateSuite.java | 64 +++++++------------ .../spark/streaming/JavaReceiverAPISuite.java | 14 ++-- 24 files changed, 123 insertions(+), 137 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/Smuggle.scala b/core/src/test/scala/org/apache/spark/Smuggle.scala index 01694a6e6f741..9f0a1b4c25dd1 100644 --- a/core/src/test/scala/org/apache/spark/Smuggle.scala +++ b/core/src/test/scala/org/apache/spark/Smuggle.scala @@ -21,6 +21,7 @@ import java.util.UUID import java.util.concurrent.locks.ReentrantReadWriteLock import scala.collection.mutable +import scala.language.implicitConversions /** * Utility wrapper to "smuggle" objects into tasks while bypassing serialization. diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaBinaryClassificationMetricsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaBinaryClassificationMetricsExample.java index 779fac01c4be0..3d8babba04a53 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaBinaryClassificationMetricsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaBinaryClassificationMetricsExample.java @@ -56,6 +56,7 @@ public static void main(String[] args) { // Compute raw scores on the test set. JavaRDD> predictionAndLabels = test.map( new Function>() { + @Override public Tuple2 call(LabeledPoint p) { Double prediction = model.predict(p.features()); return new Tuple2(prediction, p.label()); @@ -88,6 +89,7 @@ public Tuple2 call(LabeledPoint p) { // Thresholds JavaRDD thresholds = precision.map( new Function, Double>() { + @Override public Double call(Tuple2 t) { return new Double(t._1().toString()); } @@ -106,8 +108,7 @@ public Double call(Tuple2 t) { // Save and load model model.save(sc, "target/tmp/LogisticRegressionModel"); - LogisticRegressionModel sameModel = LogisticRegressionModel.load(sc, - "target/tmp/LogisticRegressionModel"); + LogisticRegressionModel.load(sc, "target/tmp/LogisticRegressionModel"); // $example off$ } } diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java index 47ab3fc358246..4ad2104763330 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java @@ -41,6 +41,7 @@ public static void main(String[] args) { JavaRDD data = sc.textFile(path); JavaRDD ratings = data.map( new Function() { + @Override public Rating call(String line) { String[] parts = line.split("::"); return new Rating(Integer.parseInt(parts[0]), Integer.parseInt(parts[1]), Double @@ -57,13 +58,14 @@ public Rating call(String line) { JavaRDD> userRecs = model.recommendProductsForUsers(10).toJavaRDD(); JavaRDD> userRecsScaled = userRecs.map( new Function, Tuple2>() { + @Override public Tuple2 call(Tuple2 t) { Rating[] scaledRatings = new Rating[t._2().length]; for (int i = 0; i < scaledRatings.length; i++) { double newRating = Math.max(Math.min(t._2()[i].rating(), 1.0), 0.0); scaledRatings[i] = new Rating(t._2()[i].user(), t._2()[i].product(), newRating); } - return new Tuple2(t._1(), scaledRatings); + return new Tuple2<>(t._1(), scaledRatings); } } ); @@ -72,6 +74,7 @@ public Tuple2 call(Tuple2 t) { // Map ratings to 1 or 0, 1 indicating a movie that should be recommended JavaRDD binarizedRatings = ratings.map( new Function() { + @Override public Rating call(Rating r) { double binaryRating; if (r.rating() > 0.0) { @@ -87,6 +90,7 @@ public Rating call(Rating r) { // Group ratings by common user JavaPairRDD> userMovies = binarizedRatings.groupBy( new Function() { + @Override public Object call(Rating r) { return r.user(); } @@ -96,8 +100,9 @@ public Object call(Rating r) { // Get true relevant documents from all user ratings JavaPairRDD> userMoviesList = userMovies.mapValues( new Function, List>() { + @Override public List call(Iterable docs) { - List products = new ArrayList(); + List products = new ArrayList<>(); for (Rating r : docs) { if (r.rating() > 0.0) { products.add(r.product()); @@ -111,8 +116,9 @@ public List call(Iterable docs) { // Extract the product id from each recommendation JavaPairRDD> userRecommendedList = userRecommended.mapValues( new Function>() { + @Override public List call(Rating[] docs) { - List products = new ArrayList(); + List products = new ArrayList<>(); for (Rating r : docs) { products.add(r.product()); } @@ -124,7 +130,7 @@ public List call(Rating[] docs) { userRecommendedList).values(); // Instantiate the metrics object - RankingMetrics metrics = RankingMetrics.of(relevantDocs); + RankingMetrics metrics = RankingMetrics.of(relevantDocs); // Precision and NDCG at k Integer[] kVector = {1, 3, 5}; @@ -139,6 +145,7 @@ public List call(Rating[] docs) { // Evaluate the model using numerical ratings and regression metrics JavaRDD> userProducts = ratings.map( new Function>() { + @Override public Tuple2 call(Rating r) { return new Tuple2(r.user(), r.product()); } @@ -147,18 +154,20 @@ public Tuple2 call(Rating r) { JavaPairRDD, Object> predictions = JavaPairRDD.fromJavaRDD( model.predict(JavaRDD.toRDD(userProducts)).toJavaRDD().map( new Function, Object>>() { + @Override public Tuple2, Object> call(Rating r) { return new Tuple2, Object>( - new Tuple2(r.user(), r.product()), r.rating()); + new Tuple2<>(r.user(), r.product()), r.rating()); } } )); JavaRDD> ratesAndPreds = JavaPairRDD.fromJavaRDD(ratings.map( new Function, Object>>() { + @Override public Tuple2, Object> call(Rating r) { return new Tuple2, Object>( - new Tuple2(r.user(), r.product()), r.rating()); + new Tuple2<>(r.user(), r.product()), r.rating()); } } )).join(predictions).values(); diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java index 90d473703ec5a..bc963a02be608 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java @@ -36,6 +36,7 @@ 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.api.java.function.VoidFunction2; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.streaming.Durations; import org.apache.spark.streaming.Time; @@ -154,9 +155,9 @@ public Integer call(Integer i1, Integer i2) { } }); - wordCounts.foreachRDD(new Function2, Time, Void>() { + wordCounts.foreachRDD(new VoidFunction2, Time>() { @Override - public Void call(JavaPairRDD rdd, Time time) throws IOException { + public void call(JavaPairRDD rdd, Time time) throws IOException { // Get or register the blacklist Broadcast final Broadcast> blacklist = JavaWordBlacklist.getInstance(new JavaSparkContext(rdd.context())); // Get or register the droppedWordsCounter Accumulator @@ -164,7 +165,7 @@ public Void call(JavaPairRDD rdd, Time time) throws IOException // Use blacklist to drop words and use droppedWordsCounter to count them String counts = rdd.filter(new Function, Boolean>() { @Override - public Boolean call(Tuple2 wordCount) throws Exception { + public Boolean call(Tuple2 wordCount) { if (blacklist.value().contains(wordCount._1())) { droppedWordsCounter.add(wordCount._2()); return false; @@ -178,7 +179,6 @@ public Boolean call(Tuple2 wordCount) throws Exception { System.out.println("Dropped " + droppedWordsCounter.value() + " word(s) totally"); System.out.println("Appending to " + outputFile.getAbsolutePath()); Files.append(output + "\n", outputFile, Charset.defaultCharset()); - return null; } }); diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java index 3515d7be45d37..084f68a8be437 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java @@ -26,7 +26,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.VoidFunction2; import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.DataFrame; import org.apache.spark.api.java.StorageLevels; @@ -78,13 +78,14 @@ public Iterable call(String x) { }); // Convert RDDs of the words DStream to DataFrame and run SQL query - words.foreachRDD(new Function2, Time, Void>() { + words.foreachRDD(new VoidFunction2, Time>() { @Override - public Void call(JavaRDD rdd, Time time) { + public void call(JavaRDD rdd, Time time) { SQLContext sqlContext = JavaSQLContextSingleton.getInstance(rdd.context()); // Convert JavaRDD[String] to JavaRDD[bean class] to DataFrame JavaRDD rowRDD = rdd.map(new Function() { + @Override public JavaRecord call(String word) { JavaRecord record = new JavaRecord(); record.setWord(word); @@ -101,7 +102,6 @@ public JavaRecord call(String word) { sqlContext.sql("select word, count(*) as total from words group by word"); System.out.println("========= " + time + "========="); wordCountsDataFrame.show(); - return null; } }); diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaTwitterHashTagJoinSentiments.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaTwitterHashTagJoinSentiments.java index 030ee30b93381..d869768026ae3 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaTwitterHashTagJoinSentiments.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaTwitterHashTagJoinSentiments.java @@ -17,13 +17,13 @@ package org.apache.spark.examples.streaming; -import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.function.FlatMapFunction; 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.api.java.function.VoidFunction; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; @@ -33,8 +33,6 @@ import scala.Tuple2; import twitter4j.Status; -import java.io.IOException; -import java.net.URI; import java.util.Arrays; import java.util.List; @@ -44,7 +42,7 @@ */ public class JavaTwitterHashTagJoinSentiments { - public static void main(String[] args) throws IOException { + public static void main(String[] args) { if (args.length < 4) { System.err.println("Usage: JavaTwitterHashTagJoinSentiments " + " []"); @@ -79,7 +77,7 @@ public Iterable call(Status s) { JavaDStream hashTags = words.filter(new Function() { @Override - public Boolean call(String word) throws Exception { + public Boolean call(String word) { return word.startsWith("#"); } }); @@ -91,8 +89,7 @@ public Boolean call(String word) throws Exception { @Override public Tuple2 call(String line) { String[] columns = line.split("\t"); - return new Tuple2(columns[0], - Double.parseDouble(columns[1])); + return new Tuple2<>(columns[0], Double.parseDouble(columns[1])); } }); @@ -101,7 +98,7 @@ public Tuple2 call(String line) { @Override public Tuple2 call(String s) { // leave out the # character - return new Tuple2(s.substring(1), 1); + return new Tuple2<>(s.substring(1), 1); } }); @@ -120,9 +117,8 @@ public Integer call(Integer a, Integer b) { hashTagTotals.transformToPair(new Function, JavaPairRDD>>() { @Override - public JavaPairRDD> call(JavaPairRDD topicCount) - throws Exception { + public JavaPairRDD> call( + JavaPairRDD topicCount) { return wordSentiments.join(topicCount); } }); @@ -131,9 +127,9 @@ public JavaPairRDD> call(JavaPairRDD>, String, Double>() { @Override public Tuple2 call(Tuple2> topicAndTuplePair) throws Exception { + Tuple2> topicAndTuplePair) { Tuple2 happinessAndCount = topicAndTuplePair._2(); - return new Tuple2(topicAndTuplePair._1(), + return new Tuple2<>(topicAndTuplePair._1(), happinessAndCount._1() * happinessAndCount._2()); } }); @@ -141,9 +137,8 @@ public Tuple2 call(Tuple2 happinessTopicPairs = topicHappiness.mapToPair( new PairFunction, Double, String>() { @Override - public Tuple2 call(Tuple2 topicHappiness) - throws Exception { - return new Tuple2(topicHappiness._2(), + public Tuple2 call(Tuple2 topicHappiness) { + return new Tuple2<>(topicHappiness._2(), topicHappiness._1()); } }); @@ -151,17 +146,17 @@ public Tuple2 call(Tuple2 topicHappiness) JavaPairDStream happiest10 = happinessTopicPairs.transformToPair( new Function, JavaPairRDD>() { @Override - public JavaPairRDD call(JavaPairRDD happinessAndTopics) throws Exception { + public JavaPairRDD call( + JavaPairRDD happinessAndTopics) { return happinessAndTopics.sortByKey(false); } } ); // Print hash tags with the most positive sentiment values - happiest10.foreachRDD(new Function, Void>() { + happiest10.foreachRDD(new VoidFunction>() { @Override - public Void call(JavaPairRDD happinessTopicPairs) throws Exception { + public void call(JavaPairRDD happinessTopicPairs) { List> topList = happinessTopicPairs.take(10); System.out.println( String.format("\nHappiest topics in last 10 seconds (%s total):", @@ -170,7 +165,6 @@ public Void call(JavaPairRDD happinessTopicPairs) throws Excepti System.out.println( String.format("%s (%s happiness)", pair._2(), pair._1())); } - return null; } }); 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 04dec57b71e16..e4486b949fb3e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -74,7 +74,7 @@ object SparkHdfsLR { val conf = new Configuration() val sc = new SparkContext(sparkConf) val lines = sc.textFile(inputPath) - val points = lines.map(parsePoint _).cache() + val points = lines.map(parsePoint).cache() val ITERATIONS = args(1).toInt // Initialize w to a random value 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 ddc99d3f90690..8b739c9d7c1db 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala @@ -71,7 +71,7 @@ object SparkTachyonHdfsLR { val conf = new Configuration() val sc = new SparkContext(sparkConf) val lines = sc.textFile(inputPath) - val points = lines.map(parsePoint _).persist(StorageLevel.OFF_HEAP) + val points = lines.map(parsePoint).persist(StorageLevel.OFF_HEAP) val ITERATIONS = args(1).toInt // Initialize w to a random value diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java index fbdfbf7e509b3..4891e4f4a17bc 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -35,6 +35,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.streaming.Durations; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; @@ -130,17 +131,15 @@ public String call(MessageAndMetadata msgAndMd) { JavaDStream unifiedStream = stream1.union(stream2); final Set result = Collections.synchronizedSet(new HashSet()); - unifiedStream.foreachRDD( - new Function, Void>() { + unifiedStream.foreachRDD(new VoidFunction>() { @Override - public Void call(JavaRDD rdd) { + public void call(JavaRDD rdd) { result.addAll(rdd.collect()); for (OffsetRange o : offsetRanges.get()) { System.out.println( o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset() ); } - return null; } } ); 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 1e69de46cd35d..617c92a008fc5 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 @@ -31,6 +31,7 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; @@ -103,10 +104,9 @@ public String call(Tuple2 tuple2) { } ); - words.countByValue().foreachRDD( - new Function, Void>() { + words.countByValue().foreachRDD(new VoidFunction>() { @Override - public Void call(JavaPairRDD rdd) { + public void call(JavaPairRDD rdd) { List> ret = rdd.collect(); for (Tuple2 r : ret) { if (result.containsKey(r._1())) { @@ -115,8 +115,6 @@ public Void call(JavaPairRDD rdd) { result.put(r._1(), r._2()); } } - - return null; } } ); diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala index 6fe24fe81165b..78263f9dca65c 100644 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala +++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala @@ -137,8 +137,8 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun // Verify that the generated KinesisBackedBlockRDD has the all the right information val blockInfos = Seq(blockInfo1, blockInfo2) val nonEmptyRDD = kinesisStream.createBlockRDD(time, blockInfos) - nonEmptyRDD shouldBe a [KinesisBackedBlockRDD[Array[Byte]]] - val kinesisRDD = nonEmptyRDD.asInstanceOf[KinesisBackedBlockRDD[Array[Byte]]] + nonEmptyRDD shouldBe a [KinesisBackedBlockRDD[_]] + val kinesisRDD = nonEmptyRDD.asInstanceOf[KinesisBackedBlockRDD[_]] assert(kinesisRDD.regionName === dummyRegionName) assert(kinesisRDD.endpointUrl === dummyEndpointUrl) assert(kinesisRDD.retryTimeoutMs === batchDuration.milliseconds) @@ -203,7 +203,7 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun Seconds(10), StorageLevel.MEMORY_ONLY, addFive, awsCredentials.getAWSAccessKeyId, awsCredentials.getAWSSecretKey) - stream shouldBe a [ReceiverInputDStream[Int]] + stream shouldBe a [ReceiverInputDStream[_]] val collected = new mutable.HashSet[Int] with mutable.SynchronizedSet[Int] stream.foreachRDD { rdd => @@ -272,7 +272,7 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun times.foreach { time => val (arrayOfSeqNumRanges, data) = collectedData(time) val rdd = recoveredKinesisStream.getOrCompute(time).get.asInstanceOf[RDD[Array[Byte]]] - rdd shouldBe a [KinesisBackedBlockRDD[Array[Byte]]] + rdd shouldBe a [KinesisBackedBlockRDD[_]] // Verify the recovered sequence ranges val kRdd = rdd.asInstanceOf[KinesisBackedBlockRDD[Array[Byte]]] 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 e47c4db62955d..ca11ede4ccd47 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 @@ -20,7 +20,7 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.ArrayBuffer import org.apache.spark.Logging -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.{axpy, scal} import org.apache.spark.mllib.util.MLUtils @@ -107,7 +107,7 @@ class KMeans private ( * Number of runs of the algorithm to execute in parallel. */ @Since("1.4.0") - @deprecated("Support for runs is deprecated. This param will have no effect in 1.7.0.", "1.6.0") + @deprecated("Support for runs is deprecated. This param will have no effect in 2.0.0.", "1.6.0") def getRuns: Int = runs /** @@ -117,7 +117,7 @@ class KMeans private ( * return the best clustering found over any run. Default: 1. */ @Since("0.8.0") - @deprecated("Support for runs is deprecated. This param will have no effect in 1.7.0.", "1.6.0") + @deprecated("Support for runs is deprecated. This param will have no effect in 2.0.0.", "1.6.0") def setRuns(runs: Int): this.type = { if (runs <= 0) { throw new IllegalArgumentException("Number of runs must be positive") @@ -431,7 +431,7 @@ class KMeans private ( val rs = (0 until runs).filter { r => rand.nextDouble() < 2.0 * c(r) * k / sumCosts(r) } - if (rs.length > 0) Some(p, rs) else None + if (rs.length > 0) Some((p, rs)) else None } }.collect() mergeNewCenters() 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 271dda4662e0d..a6631ed7ebd6f 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 @@ -56,10 +56,10 @@ void validatePrediction( double matchThreshold, boolean implicitPrefs, DoubleMatrix truePrefs) { - List> localUsersProducts = new ArrayList(users * products); + List> localUsersProducts = new ArrayList<>(users * products); for (int u=0; u < users; ++u) { for (int p=0; p < products; ++p) { - localUsersProducts.add(new Tuple2(u, p)); + localUsersProducts.add(new Tuple2<>(u, p)); } } JavaPairRDD usersProducts = sc.parallelizePairs(localUsersProducts); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java index 32c2f4f3395b7..3db9b39e740e7 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java @@ -36,11 +36,11 @@ public class JavaIsotonicRegressionSuite implements Serializable { private transient JavaSparkContext sc; - private List> generateIsotonicInput(double[] labels) { - ArrayList> input = new ArrayList(labels.length); + private static List> generateIsotonicInput(double[] labels) { + List> input = new ArrayList<>(labels.length); for (int i = 1; i <= labels.length; i++) { - input.add(new Tuple3(labels[i-1], (double) i, 1d)); + input.add(new Tuple3<>(labels[i-1], (double) i, 1.0)); } return input; @@ -70,7 +70,7 @@ public void testIsotonicRegressionJavaRDD() { runIsotonicRegression(new double[]{1, 2, 3, 3, 1, 6, 7, 8, 11, 9, 10, 12}); Assert.assertArrayEquals( - new double[] {1, 2, 7d/3, 7d/3, 6, 7, 8, 10, 10, 12}, model.predictions(), 1e-14); + new double[] {1, 2, 7.0/3, 7.0/3, 6, 7, 8, 10, 10, 12}, model.predictions(), 1.0e-14); } @Test @@ -81,10 +81,10 @@ public void testIsotonicRegressionPredictionsJavaRDD() { JavaDoubleRDD testRDD = sc.parallelizeDoubles(Arrays.asList(0.0, 1.0, 9.5, 12.0, 13.0)); List predictions = model.predict(testRDD).collect(); - Assert.assertTrue(predictions.get(0) == 1d); - Assert.assertTrue(predictions.get(1) == 1d); - Assert.assertTrue(predictions.get(2) == 10d); - Assert.assertTrue(predictions.get(3) == 12d); - Assert.assertTrue(predictions.get(4) == 12d); + Assert.assertEquals(1.0, predictions.get(0).doubleValue(), 1.0e-14); + Assert.assertEquals(1.0, predictions.get(1).doubleValue(), 1.0e-14); + Assert.assertEquals(10.0, predictions.get(2).doubleValue(), 1.0e-14); + Assert.assertEquals(12.0, predictions.get(3).doubleValue(), 1.0e-14); + Assert.assertEquals(12.0, predictions.get(4).doubleValue(), 1.0e-14); } } diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 48daa87e82d13..d22a7f4c3b167 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -173,7 +173,7 @@ def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||" """Train a k-means clustering model.""" if runs != 1: warnings.warn( - "Support for runs is deprecated in 1.6.0. This param will have no effect in 1.7.0.") + "Support for runs is deprecated in 1.6.0. This param will have no effect in 2.0.0.") clusterInitialModel = [] if initialModel is not None: if not isinstance(initialModel, KMeansModel): diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index f869a96edb1ce..e028d22a54ba0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -57,8 +57,8 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { (result, expected) match { case (result: Array[Byte], expected: Array[Byte]) => java.util.Arrays.equals(result, expected) - case (result: Double, expected: Spread[Double]) => - expected.isWithin(result) + case (result: Double, expected: Spread[Double @unchecked]) => + expected.asInstanceOf[Spread[Double]].isWithin(result) case _ => result == expected } } @@ -275,8 +275,8 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { (result, expected) match { case (result: Array[Byte], expected: Array[Byte]) => java.util.Arrays.equals(result, expected) - case (result: Double, expected: Spread[Double]) => - expected.isWithin(result) + case (result: Double, expected: Spread[Double @unchecked]) => + expected.asInstanceOf[Spread[Double]].isWithin(result) case (result: Double, expected: Double) if result.isNaN && expected.isNaN => true case (result: Float, expected: Float) if result.isNaN && expected.isNaN => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index d5f1c4d74efcf..6745b4b6c3c67 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -384,9 +384,6 @@ class DateTimeUtilsSuite extends SparkFunSuite { Timestamp.valueOf("1700-02-28 12:14:50.123456")).foreach { t => val us = fromJavaTimestamp(t) assert(toJavaTimestamp(us) === t) - assert(getHours(us) === t.getHours) - assert(getMinutes(us) === t.getMinutes) - assert(getSeconds(us) === t.getSeconds) } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java index f8e32d60a489a..6bcd155ccdc49 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java @@ -21,6 +21,7 @@ import java.io.ByteArrayInputStream; import java.io.File; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -62,7 +63,7 @@ import org.apache.spark.sql.types.StructType; /** - * Base class for custom RecordReaaders for Parquet that directly materialize to `T`. + * Base class for custom RecordReaders for Parquet that directly materialize to `T`. * This class handles computing row groups, filtering on them, setting up the column readers, * etc. * This is heavily based on parquet-mr's RecordReader. @@ -83,6 +84,7 @@ public abstract class SpecificParquetRecordReaderBase extends RecordReader fileMetadata = footer.getFileMetaData().getKeyValueMetaData(); - ReadSupport readSupport = getReadSupportInstance( - (Class>) getReadSupportClass(configuration)); + ReadSupport readSupport = getReadSupportInstance(getReadSupportClass(configuration)); ReadSupport.ReadContext readContext = readSupport.init(new InitContext( taskAttemptContext.getConfiguration(), toSetMultiMap(fileMetadata), fileSchema)); this.requestedSchema = readContext.getRequestedSchema(); @@ -282,8 +283,9 @@ private static Map> toSetMultiMap(Map map) { return Collections.unmodifiableMap(setMultiMap); } - private static Class getReadSupportClass(Configuration configuration) { - return ConfigurationUtil.getClassFromConfig(configuration, + @SuppressWarnings("unchecked") + private Class> getReadSupportClass(Configuration configuration) { + return (Class>) ConfigurationUtil.getClassFromConfig(configuration, ParquetInputFormat.READ_SUPPORT_CLASS, ReadSupport.class); } @@ -294,10 +296,9 @@ private static Class getReadSupportClass(Configuration configuration) { private static ReadSupport getReadSupportInstance( Class> readSupportClass){ try { - return readSupportClass.newInstance(); - } catch (InstantiationException e) { - throw new BadConfigurationException("could not instantiate read support class", e); - } catch (IllegalAccessException e) { + return readSupportClass.getConstructor().newInstance(); + } catch (InstantiationException | IllegalAccessException | + NoSuchMethodException | InvocationTargetException e) { throw new BadConfigurationException("could not instantiate read support class", e); } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 076db0c08dee0..eb4efcd1d4e41 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -580,7 +580,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { ) } - test("sparkPartitionId") { + test("spark_partition_id") { // Make sure we have 2 partitions, each with 2 records. val df = sparkContext.parallelize(Seq[Int](), 2).mapPartitions { _ => Iterator(Tuple1(1), Tuple1(2)) @@ -591,7 +591,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { ) } - test("InputFileName") { + test("input_file_name") { withTempPath { dir => val data = sparkContext.parallelize(0 to 10).toDF("id") data.write.parquet(dir.getCanonicalPath) 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 0e60573dc6b2c..fac26bd0c0269 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql import java.util.{Locale, TimeZone} import scala.collection.JavaConverters._ +import scala.util.control.NonFatal import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.ImperativeAggregate @@ -206,7 +207,7 @@ abstract class QueryTest extends PlanTest { val jsonString = try { logicalPlan.toJSON } catch { - case e => + case NonFatal(e) => fail( s""" |Failed to parse logical plan to JSON: @@ -231,7 +232,7 @@ abstract class QueryTest extends PlanTest { val jsonBackPlan = try { TreeNode.fromJSON[LogicalPlan](jsonString, sqlContext.sparkContext) } catch { - case e => + case NonFatal(e) => fail( s""" |Failed to rebuild the logical plan from JSON: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala index 97cba1e349e8f..1529313dfbd51 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala @@ -60,6 +60,7 @@ object ColumnarTestUtils { case MAP(_) => ArrayBasedMapData( Map(Random.nextInt() -> UTF8String.fromString(Random.nextString(Random.nextInt(32))))) + case _ => throw new IllegalArgumentException(s"Unknown column type $columnType") }).asInstanceOf[JvmType] } diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 9722c60bba1c3..ddc56fc869ae1 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -772,8 +772,8 @@ public Iterable call(String x) { @SuppressWarnings("unchecked") @Test public void testForeachRDD() { - final Accumulator accumRdd = ssc.sc().accumulator(0); - final Accumulator accumEle = ssc.sc().accumulator(0); + final Accumulator accumRdd = ssc.sparkContext().accumulator(0); + final Accumulator accumEle = ssc.sparkContext().accumulator(0); List> inputData = Arrays.asList( Arrays.asList(1,1,1), Arrays.asList(1,1,1)); diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java index bc4bc2eb42231..20e2a1c3d5c31 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java @@ -18,6 +18,7 @@ package org.apache.spark.streaming; import java.io.Serializable; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -26,10 +27,10 @@ import scala.Tuple2; import com.google.common.base.Optional; -import com.google.common.collect.Lists; import com.google.common.collect.Sets; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.util.ManualClock; import org.junit.Assert; @@ -51,10 +52,8 @@ public void testAPI() { JavaPairRDD initialRDD = null; JavaPairDStream wordsDstream = null; - final Function4, State, Optional> - mappingFunc = + Function4, State, Optional> mappingFunc = new Function4, State, Optional>() { - @Override public Optional call( Time time, String word, Optional one, State state) { @@ -76,11 +75,10 @@ public Optional call( .partitioner(new HashPartitioner(10)) .timeout(Durations.seconds(10))); - JavaPairDStream stateSnapshots = stateDstream.stateSnapshots(); + stateDstream.stateSnapshots(); - final Function3, State, Double> mappingFunc2 = + Function3, State, Double> mappingFunc2 = new Function3, State, Double>() { - @Override public Double call(String key, Optional one, State state) { // Use all State's methods here @@ -95,13 +93,13 @@ public Double call(String key, Optional one, State state) { JavaMapWithStateDStream stateDstream2 = wordsDstream.mapWithState( - StateSpec.function(mappingFunc2) + StateSpec.function(mappingFunc2) .initialState(initialRDD) .numPartitions(10) .partitioner(new HashPartitioner(10)) .timeout(Durations.seconds(10))); - JavaPairDStream stateSnapshots2 = stateDstream2.stateSnapshots(); + stateDstream2.stateSnapshots(); } @Test @@ -126,33 +124,21 @@ public void testBasicFunction() { Collections.emptySet() ); + @SuppressWarnings("unchecked") List>> stateData = Arrays.asList( Collections.>emptySet(), - Sets.newHashSet(new Tuple2("a", 1)), - Sets.newHashSet(new Tuple2("a", 2), new Tuple2("b", 1)), - Sets.newHashSet( - new Tuple2("a", 3), - new Tuple2("b", 2), - new Tuple2("c", 1)), - Sets.newHashSet( - new Tuple2("a", 4), - new Tuple2("b", 3), - new Tuple2("c", 1)), - Sets.newHashSet( - new Tuple2("a", 5), - new Tuple2("b", 3), - new Tuple2("c", 1)), - Sets.newHashSet( - new Tuple2("a", 5), - new Tuple2("b", 3), - new Tuple2("c", 1)) + Sets.newHashSet(new Tuple2<>("a", 1)), + Sets.newHashSet(new Tuple2<>("a", 2), new Tuple2<>("b", 1)), + Sets.newHashSet(new Tuple2<>("a", 3), new Tuple2<>("b", 2), new Tuple2<>("c", 1)), + Sets.newHashSet(new Tuple2<>("a", 4), new Tuple2<>("b", 3), new Tuple2<>("c", 1)), + Sets.newHashSet(new Tuple2<>("a", 5), new Tuple2<>("b", 3), new Tuple2<>("c", 1)), + Sets.newHashSet(new Tuple2<>("a", 5), new Tuple2<>("b", 3), new Tuple2<>("c", 1)) ); Function3, State, Integer> mappingFunc = new Function3, State, Integer>() { - @Override - public Integer call(String key, Optional value, State state) throws Exception { + public Integer call(String key, Optional value, State state) { int sum = value.or(0) + (state.exists() ? state.get() : 0); state.update(sum); return sum; @@ -160,7 +146,7 @@ public Integer call(String key, Optional value, State state) t }; testOperation( inputData, - StateSpec.function(mappingFunc), + StateSpec.function(mappingFunc), outputData, stateData); } @@ -175,27 +161,25 @@ private void testOperation( JavaMapWithStateDStream mapWithStateDStream = JavaPairDStream.fromJavaDStream(inputStream.map(new Function>() { @Override - public Tuple2 call(K x) throws Exception { - return new Tuple2(x, 1); + public Tuple2 call(K x) { + return new Tuple2<>(x, 1); } })).mapWithState(mapWithStateSpec); final List> collectedOutputs = - Collections.synchronizedList(Lists.>newArrayList()); - mapWithStateDStream.foreachRDD(new Function, Void>() { + Collections.synchronizedList(new ArrayList>()); + mapWithStateDStream.foreachRDD(new VoidFunction>() { @Override - public Void call(JavaRDD rdd) throws Exception { + public void call(JavaRDD rdd) { collectedOutputs.add(Sets.newHashSet(rdd.collect())); - return null; } }); final List>> collectedStateSnapshots = - Collections.synchronizedList(Lists.>>newArrayList()); - mapWithStateDStream.stateSnapshots().foreachRDD(new Function, Void>() { + Collections.synchronizedList(new ArrayList>>()); + mapWithStateDStream.stateSnapshots().foreachRDD(new VoidFunction>() { @Override - public Void call(JavaPairRDD rdd) throws Exception { + public void call(JavaPairRDD rdd) { collectedStateSnapshots.add(Sets.newHashSet(rdd.collect())); - return null; } }); BatchCounter batchCounter = new BatchCounter(ssc.ssc()); diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java index 7a8ef9d14784c..d09258e0e4a85 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java @@ -18,13 +18,14 @@ package org.apache.spark.streaming; import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import static org.junit.Assert.*; import com.google.common.io.Closeables; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -68,12 +69,11 @@ public String call(String v1) { return v1 + "."; } }); - mapped.foreachRDD(new Function, Void>() { + mapped.foreachRDD(new VoidFunction>() { @Override - public Void call(JavaRDD rdd) { + public void call(JavaRDD rdd) { long count = rdd.count(); dataCounter.addAndGet(count); - return null; } }); @@ -90,7 +90,7 @@ public Void call(JavaRDD rdd) { Thread.sleep(100); } ssc.stop(); - assertTrue(dataCounter.get() > 0); + Assert.assertTrue(dataCounter.get() > 0); } finally { server.stop(); } @@ -98,8 +98,8 @@ public Void call(JavaRDD rdd) { private static class JavaSocketReceiver extends Receiver { - String host = null; - int port = -1; + private String host = null; + private int port = -1; JavaSocketReceiver(String host_ , int port_) { super(StorageLevel.MEMORY_AND_DISK()); From cfe1ba56e4ab281a9e8eaf419fb7429f93c7a0ce Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 8 Jan 2016 09:50:41 -0800 Subject: [PATCH 038/110] [SPARK-12687] [SQL] Support from clause surrounded by `()`. JIRA: https://issues.apache.org/jira/browse/SPARK-12687 Some queries such as `(select 1 as a) union (select 2 as a)` can't work. This patch fixes it. Author: Liang-Chi Hsieh Closes #10660 from viirya/fix-union. --- .../sql/catalyst/parser/FromClauseParser.g | 2 +- .../sql/catalyst/parser/SparkSqlParser.g | 21 ++++++++++++++++++- .../spark/sql/catalyst/CatalystQlSuite.scala | 4 ++++ 3 files changed, 25 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/FromClauseParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/FromClauseParser.g index ba6cfc60f045f..972c52e3ffcec 100644 --- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/FromClauseParser.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/FromClauseParser.g @@ -151,8 +151,8 @@ fromSource @after { gParent.popMsg(state); } : (LPAREN KW_VALUES) => fromSource0 - | (LPAREN) => LPAREN joinSource RPAREN -> joinSource | fromSource0 + | (LPAREN joinSource) => LPAREN joinSource RPAREN -> joinSource ; diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g index cf8a56566d32d..b04bb677774c5 100644 --- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g @@ -2216,6 +2216,8 @@ regularBody[boolean topLevel] selectStatement[boolean topLevel] : ( + ( + LPAREN s=selectClause f=fromClause? w=whereClause? @@ -2227,6 +2229,20 @@ selectStatement[boolean topLevel] sort=sortByClause? win=window_clause? l=limitClause? + RPAREN + | + s=selectClause + f=fromClause? + w=whereClause? + g=groupByClause? + h=havingClause? + o=orderByClause? + c=clusterByClause? + d=distributeByClause? + sort=sortByClause? + win=window_clause? + l=limitClause? + ) -> ^(TOK_QUERY $f? ^(TOK_INSERT ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE)) $s $w? $g? $h? $o? $c? $d? $sort? $win? $l?)) @@ -2241,7 +2257,10 @@ selectStatement[boolean topLevel] setOpSelectStatement[CommonTree t, boolean topLevel] : - (u=setOperator b=simpleSelectStatement + (( + u=setOperator LPAREN b=simpleSelectStatement RPAREN + | + u=setOperator b=simpleSelectStatement) -> {$setOpSelectStatement.tree != null && $u.tree.getType()==SparkSqlParser.TOK_UNIONDISTINCT}? ^(TOK_QUERY ^(TOK_FROM diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala index 0fee97fb0718c..30978d9b49e2b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala @@ -28,5 +28,9 @@ class CatalystQlSuite extends PlanTest { paresr.createPlan("select * from t1 union select * from t2") paresr.createPlan("select * from t1 except select * from t2") paresr.createPlan("select * from t1 intersect select * from t2") + paresr.createPlan("(select * from t1) union all (select * from t2)") + paresr.createPlan("(select * from t1) union distinct (select * from t2)") + paresr.createPlan("(select * from t1) union (select * from t2)") + paresr.createPlan("select * from ((select * from t1) union (select * from t2)) t") } } From ea104b8f1ce8aa109d1b16b696a61a47df6283b2 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Fri, 8 Jan 2016 11:08:45 -0800 Subject: [PATCH 039/110] [SPARK-12701][CORE] FileAppender should use join to ensure writing thread completion Changed Logging FileAppender to use join in `awaitTermination` to ensure that thread is properly finished before returning. Author: Bryan Cutler Closes #10654 from BryanCutler/fileAppender-join-thread-SPARK-12701. --- .../org/apache/spark/util/logging/FileAppender.scala | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala index 14b6ba4af489a..58c8560a3d049 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala @@ -29,7 +29,6 @@ private[spark] class FileAppender(inputStream: InputStream, file: File, bufferSi extends Logging { @volatile private var outputStream: FileOutputStream = null @volatile private var markedForStop = false // has the appender been asked to stopped - @volatile private var stopped = false // has the appender stopped // Thread that reads the input stream and writes to file private val writingThread = new Thread("File appending thread for " + file) { @@ -47,11 +46,7 @@ private[spark] class FileAppender(inputStream: InputStream, file: File, bufferSi * or because of any error in appending */ def awaitTermination() { - synchronized { - if (!stopped) { - wait() - } - } + writingThread.join() } /** Stop the appender */ @@ -77,10 +72,6 @@ private[spark] class FileAppender(inputStream: InputStream, file: File, bufferSi logError(s"Error writing stream to file $file", e) } finally { closeFile() - synchronized { - stopped = true - notifyAll() - } } } From 00d9261724feb48d358679efbae6889833e893e0 Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Fri, 8 Jan 2016 11:38:46 -0800 Subject: [PATCH 040/110] [DOCUMENTATION] doc fix of job scheduling spark.shuffle.service.enabled is spark application related configuration, it is not necessary to set it in yarn-site.xml Author: Jeff Zhang Closes #10657 from zjffdu/doc-fix. --- docs/job-scheduling.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 36327c6efeaf3..6c587b3f0d8db 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -91,7 +91,7 @@ pre-packaged distribution. 2. Add this jar to the classpath of all `NodeManager`s in your cluster. 3. In the `yarn-site.xml` on each node, add `spark_shuffle` to `yarn.nodemanager.aux-services`, then set `yarn.nodemanager.aux-services.spark_shuffle.class` to -`org.apache.spark.network.yarn.YarnShuffleService` and `spark.shuffle.service.enabled` to true. +`org.apache.spark.network.yarn.YarnShuffleService`. 4. Restart all `NodeManager`s in your cluster. All other relevant configurations are optional and under the `spark.dynamicAllocation.*` and From 8c70cb4c62a353bea99f37965dfc829c4accc391 Mon Sep 17 00:00:00 2001 From: Udo Klein Date: Fri, 8 Jan 2016 20:32:37 +0000 Subject: [PATCH 041/110] fixed numVertices in transitive closure example Author: Udo Klein Closes #10642 from udoklein/patch-2. --- examples/src/main/python/transitive_closure.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index 7bf5fb6ddfe29..3d61250d8b230 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -30,8 +30,8 @@ def generateGraph(): edges = set() while len(edges) < numEdges: - src = rand.randrange(0, numEdges) - dst = rand.randrange(0, numEdges) + src = rand.randrange(0, numVertices) + dst = rand.randrange(0, numVertices) if src != dst: edges.add((src, dst)) return edges From 553fd7b912a32476b481fd3f80c1d0664b6c6484 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 8 Jan 2016 14:38:19 -0600 Subject: [PATCH 042/110] =?UTF-8?q?[SPARK-12654]=20sc.wholeTextFiles=20wit?= =?UTF-8?q?h=20spark.hadoop.cloneConf=3Dtrue=20fail=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …s on secure Hadoop https://issues.apache.org/jira/browse/SPARK-12654 So the bug here is that WholeTextFileRDD.getPartitions has: val conf = getConf in getConf if the cloneConf=true it creates a new Hadoop Configuration. Then it uses that to create a new newJobContext. The newJobContext will copy credentials around, but credentials are only present in a JobConf not in a Hadoop Configuration. So basically when it is cloning the hadoop configuration its changing it from a JobConf to Configuration and dropping the credentials that were there. NewHadoopRDD just uses the conf passed in for the getPartitions (not getConf) which is why it works. Author: Thomas Graves Closes #10651 from tgravescs/SPARK-12654. --- .../main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) 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 146609ae3911a..7a1197830443f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, FileSplit} import org.apache.hadoop.mapreduce.task.{JobContextImpl, TaskAttemptContextImpl} @@ -93,7 +94,13 @@ class NewHadoopRDD[K, V]( // issues, this cloning is disabled by default. NewHadoopRDD.CONFIGURATION_INSTANTIATION_LOCK.synchronized { logDebug("Cloning Hadoop Configuration") - new Configuration(conf) + // The Configuration passed in is actually a JobConf and possibly contains credentials. + // To keep those credentials properly we have to create a new JobConf not a Configuration. + if (conf.isInstanceOf[JobConf]) { + new JobConf(conf) + } else { + new Configuration(conf) + } } } else { conf From 659fd9d04b988d48960eac4f352ca37066f43f5c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 8 Jan 2016 13:02:30 -0800 Subject: [PATCH 043/110] [SPARK-4819] Remove Guava's "Optional" from public API Replace Guava `Optional` with (an API clone of) Java 8 `java.util.Optional` (edit: and a clone of Guava `Optional`) See also https://github.com/apache/spark/pull/10512 Author: Sean Owen Closes #10513 from srowen/SPARK-4819. --- .../org/apache/spark/api/java/Optional.java | 187 ++++++++++++++++++ .../apache/spark/api/java/JavaPairRDD.scala | 2 - .../apache/spark/api/java/JavaRDDLike.scala | 4 - .../spark/api/java/JavaSparkContext.scala | 1 - .../org/apache/spark/api/java/JavaUtils.scala | 9 +- .../java/org/apache/spark/JavaAPISuite.java | 46 ++--- .../apache/spark/api/java/OptionalSuite.java | 94 +++++++++ docs/streaming-programming-guide.md | 1 - .../JavaStatefulNetworkWordCount.java | 20 +- .../java/org/apache/spark/Java8APISuite.java | 2 +- .../apache/spark/streaming/Java8APISuite.java | 1 - network/common/pom.xml | 6 - pom.xml | 11 -- project/MimaExcludes.scala | 11 +- .../apache/spark/streaming/StateSpec.scala | 12 +- .../streaming/api/java/JavaPairDStream.scala | 3 +- .../apache/spark/streaming/JavaAPISuite.java | 2 +- .../streaming/JavaMapWithStateSuite.java | 4 +- .../tools/JavaAPICompletenessChecker.scala | 2 +- 19 files changed, 333 insertions(+), 85 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/api/java/Optional.java create mode 100644 core/src/test/java/org/apache/spark/api/java/OptionalSuite.java diff --git a/core/src/main/java/org/apache/spark/api/java/Optional.java b/core/src/main/java/org/apache/spark/api/java/Optional.java new file mode 100644 index 0000000000000..ca7babc3f01c7 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/Optional.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.io.Serializable; + +import com.google.common.base.Preconditions; + +/** + *

Like {@code java.util.Optional} in Java 8, {@code scala.Option} in Scala, and + * {@code com.google.common.base.Optional} in Google Guava, this class represents a + * value of a given type that may or may not exist. It is used in methods that wish + * to optionally return a value, in preference to returning {@code null}.

+ * + *

In fact, the class here is a reimplementation of the essential API of both + * {@code java.util.Optional} and {@code com.google.common.base.Optional}. From + * {@code java.util.Optional}, it implements:

+ * + *
    + *
  • {@link #empty()}
  • + *
  • {@link #of(Object)}
  • + *
  • {@link #ofNullable(Object)}
  • + *
  • {@link #get()}
  • + *
  • {@link #orElse(Object)}
  • + *
  • {@link #isPresent()}
  • + *
+ * + *

From {@code com.google.common.base.Optional} it implements:

+ * + *
    + *
  • {@link #absent()}
  • + *
  • {@link #of(Object)}
  • + *
  • {@link #fromNullable(Object)}
  • + *
  • {@link #get()}
  • + *
  • {@link #or(Object)}
  • + *
  • {@link #orNull()}
  • + *
  • {@link #isPresent()}
  • + *
+ * + *

{@code java.util.Optional} itself is not used at this time because the + * project does not require Java 8. Using {@code com.google.common.base.Optional} + * has in the past caused serious library version conflicts with Guava that can't + * be resolved by shading. Hence this work-alike clone.

+ * + * @param type of value held inside + */ +public final class Optional implements Serializable { + + private static final Optional EMPTY = new Optional<>(); + + private final T value; + + private Optional() { + this.value = null; + } + + private Optional(T value) { + Preconditions.checkNotNull(value); + this.value = value; + } + + // java.util.Optional API (subset) + + /** + * @return an empty {@code Optional} + */ + public static Optional empty() { + @SuppressWarnings("unchecked") + Optional t = (Optional) EMPTY; + return t; + } + + /** + * @param value non-null value to wrap + * @return {@code Optional} wrapping this value + * @throws NullPointerException if value is null + */ + public static Optional of(T value) { + return new Optional<>(value); + } + + /** + * @param value value to wrap, which may be null + * @return {@code Optional} wrapping this value, which may be empty + */ + public static Optional ofNullable(T value) { + if (value == null) { + return empty(); + } else { + return of(value); + } + } + + /** + * @return the value wrapped by this {@code Optional} + * @throws NullPointerException if this is empty (contains no value) + */ + public T get() { + Preconditions.checkNotNull(value); + return value; + } + + /** + * @param other value to return if this is empty + * @return this {@code Optional}'s value if present, or else the given value + */ + public T orElse(T other) { + return value != null ? value : other; + } + + /** + * @return true iff this {@code Optional} contains a value (non-empty) + */ + public boolean isPresent() { + return value != null; + } + + // Guava API (subset) + // of(), get() and isPresent() are identically present in the Guava API + + /** + * @return an empty {@code Optional} + */ + public static Optional absent() { + return empty(); + } + + /** + * @param value value to wrap, which may be null + * @return {@code Optional} wrapping this value, which may be empty + */ + public static Optional fromNullable(T value) { + return ofNullable(value); + } + + /** + * @param other value to return if this is empty + * @return this {@code Optional}'s value if present, or else the given value + */ + public T or(T other) { + return value != null ? value : other; + } + + /** + * @return this {@code Optional}'s value if present, or else null + */ + public T orNull() { + return value; + } + + // Common methods + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof Optional)) { + return false; + } + Optional other = (Optional) obj; + return value == null ? other.value == null : value.equals(other.value); + } + + @Override + public int hashCode() { + return value == null ? 0 : value.hashCode(); + } + + @Override + public String toString() { + return value == null ? "Optional.empty" : String.format("Optional[%s]", value); + } + +} 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 59af1052ebd05..fb04472ee73fd 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 @@ -25,7 +25,6 @@ import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.ClassTag -import com.google.common.base.Optional import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{JobConf, OutputFormat} @@ -655,7 +654,6 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * keys; this also retains the original RDD's partitioning. */ def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairRDD[K, U] = { - import scala.collection.JavaConverters._ def fn: (V) => Iterable[U] = (x: V) => f.call(x).asScala implicit val ctag: ClassTag[U] = fakeClassTag fromRDD(rdd.flatMapValues(fn)) 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 242438237f987..0f8d13cf5cc2f 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 @@ -24,7 +24,6 @@ import java.util.{Comparator, Iterator => JIterator, List => JList} 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._ @@ -122,7 +121,6 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * RDD, and then flattening the results. */ def flatMap[U](f: FlatMapFunction[T, U]): JavaRDD[U] = { - import scala.collection.JavaConverters._ def fn: (T) => Iterable[U] = (x: T) => f.call(x).asScala JavaRDD.fromRDD(rdd.flatMap(fn)(fakeClassTag[U]))(fakeClassTag[U]) } @@ -132,7 +130,6 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * RDD, and then flattening the results. */ def flatMapToDouble(f: DoubleFlatMapFunction[T]): JavaDoubleRDD = { - import scala.collection.JavaConverters._ def fn: (T) => Iterable[jl.Double] = (x: T) => f.call(x).asScala new JavaDoubleRDD(rdd.flatMap(fn).map((x: jl.Double) => x.doubleValue())) } @@ -142,7 +139,6 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * RDD, and then flattening the results. */ def flatMapToPair[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairRDD[K2, V2] = { - import scala.collection.JavaConverters._ def fn: (T) => Iterable[(K2, V2)] = (x: T) => f.call(x).asScala def cm: ClassTag[(K2, V2)] = implicitly[ClassTag[(K2, V2)]] JavaPairRDD.fromRDD(rdd.flatMap(fn)(cm))(fakeClassTag[K2], fakeClassTag[V2]) 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 9990b22e14a25..01433ca2efc14 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 @@ -25,7 +25,6 @@ import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.ClassTag -import com.google.common.base.Optional import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.{InputFormat, JobConf} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} 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 b2a4d053fa650..f820401da2fc3 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 @@ -22,13 +22,12 @@ import java.util.Map.Entry import scala.collection.mutable -import com.google.common.base.Optional - private[spark] object JavaUtils { def optionToOptional[T](option: Option[T]): Optional[T] = - option match { - case Some(value) => Optional.of(value) - case None => Optional.absent() + if (option.isDefined) { + Optional.of(option.get) + } else { + Optional.empty[T] } // Workaround for SPARK-3926 / SI-8911 diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 47382e4231563..44d5cac7c2de5 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -21,7 +21,17 @@ import java.nio.channels.FileChannel; import java.nio.ByteBuffer; import java.net.URI; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.concurrent.*; import scala.Tuple2; @@ -35,7 +45,6 @@ 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; import org.apache.hadoop.io.IntWritable; @@ -49,7 +58,12 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.*; +import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.api.java.JavaFutureAction; +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.Optional; import org.apache.spark.api.java.function.*; import org.apache.spark.input.PortableDataStream; import org.apache.spark.partial.BoundedDouble; @@ -1785,32 +1799,6 @@ public void testAsyncActionErrorWrapping() throws 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. - */ - @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,1024]", "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(); - } - } - static class Class1 {} static class Class2 {} diff --git a/core/src/test/java/org/apache/spark/api/java/OptionalSuite.java b/core/src/test/java/org/apache/spark/api/java/OptionalSuite.java new file mode 100644 index 0000000000000..4b97c18198c1a --- /dev/null +++ b/core/src/test/java/org/apache/spark/api/java/OptionalSuite.java @@ -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.api.java; + +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests {@link Optional}. + */ +public class OptionalSuite { + + @Test + public void testEmpty() { + Assert.assertFalse(Optional.empty().isPresent()); + Assert.assertNull(Optional.empty().orNull()); + Assert.assertEquals("foo", Optional.empty().or("foo")); + Assert.assertEquals("foo", Optional.empty().orElse("foo")); + } + + @Test(expected = NullPointerException.class) + public void testEmptyGet() { + Optional.empty().get(); + } + + @Test + public void testAbsent() { + Assert.assertFalse(Optional.absent().isPresent()); + Assert.assertNull(Optional.absent().orNull()); + Assert.assertEquals("foo", Optional.absent().or("foo")); + Assert.assertEquals("foo", Optional.absent().orElse("foo")); + } + + @Test(expected = NullPointerException.class) + public void testAbsentGet() { + Optional.absent().get(); + } + + @Test + public void testOf() { + Assert.assertTrue(Optional.of(1).isPresent()); + Assert.assertNotNull(Optional.of(1).orNull()); + Assert.assertEquals(Integer.valueOf(1), Optional.of(1).get()); + Assert.assertEquals(Integer.valueOf(1), Optional.of(1).or(2)); + Assert.assertEquals(Integer.valueOf(1), Optional.of(1).orElse(2)); + } + + @Test(expected = NullPointerException.class) + public void testOfWithNull() { + Optional.of(null); + } + + @Test + public void testOfNullable() { + Assert.assertTrue(Optional.ofNullable(1).isPresent()); + Assert.assertNotNull(Optional.ofNullable(1).orNull()); + Assert.assertEquals(Integer.valueOf(1), Optional.ofNullable(1).get()); + Assert.assertEquals(Integer.valueOf(1), Optional.ofNullable(1).or(2)); + Assert.assertEquals(Integer.valueOf(1), Optional.ofNullable(1).orElse(2)); + Assert.assertFalse(Optional.ofNullable(null).isPresent()); + Assert.assertNull(Optional.ofNullable(null).orNull()); + Assert.assertEquals(Integer.valueOf(2), Optional.ofNullable(null).or(2)); + Assert.assertEquals(Integer.valueOf(2), Optional.ofNullable(null).orElse(2)); + } + + @Test + public void testFromNullable() { + Assert.assertTrue(Optional.fromNullable(1).isPresent()); + Assert.assertNotNull(Optional.fromNullable(1).orNull()); + Assert.assertEquals(Integer.valueOf(1), Optional.fromNullable(1).get()); + Assert.assertEquals(Integer.valueOf(1), Optional.fromNullable(1).or(2)); + Assert.assertEquals(Integer.valueOf(1), Optional.fromNullable(1).orElse(2)); + Assert.assertFalse(Optional.fromNullable(null).isPresent()); + Assert.assertNull(Optional.fromNullable(null).orNull()); + Assert.assertEquals(Integer.valueOf(2), Optional.fromNullable(null).or(2)); + Assert.assertEquals(Integer.valueOf(2), Optional.fromNullable(null).orElse(2)); + } + +} diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 1edc0fe34706b..8fd075d02b78e 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -881,7 +881,6 @@ Scala code, take a look at the example
{% highlight java %} -import com.google.common.base.Optional; Function2, Optional, Optional> updateFunction = new Function2, Optional, Optional>() { @Override public Optional call(List values, Optional state) { diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java index 14997c64d505e..f52cc7c20576b 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java @@ -23,17 +23,14 @@ import scala.Tuple2; -import com.google.common.base.Optional; -import com.google.common.collect.Lists; - import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.*; import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.Optional; import org.apache.spark.api.java.StorageLevels; import org.apache.spark.streaming.Durations; import org.apache.spark.streaming.State; import org.apache.spark.streaming.StateSpec; -import org.apache.spark.streaming.Time; import org.apache.spark.streaming.api.java.*; /** @@ -67,8 +64,8 @@ public static void main(String[] args) { // Initial state RDD input to mapWithState @SuppressWarnings("unchecked") - List> tuples = Arrays.asList(new Tuple2("hello", 1), - new Tuple2("world", 1)); + List> tuples = + Arrays.asList(new Tuple2<>("hello", 1), new Tuple2<>("world", 1)); JavaPairRDD initialRDD = ssc.sparkContext().parallelizePairs(tuples); JavaReceiverInputDStream lines = ssc.socketTextStream( @@ -77,7 +74,7 @@ public static void main(String[] args) { JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { - return Lists.newArrayList(SPACE.split(x)); + return Arrays.asList(SPACE.split(x)); } }); @@ -85,18 +82,17 @@ public Iterable call(String x) { new PairFunction() { @Override public Tuple2 call(String s) { - return new Tuple2(s, 1); + return new Tuple2<>(s, 1); } }); // Update the cumulative count function - final Function3, State, Tuple2> mappingFunc = + Function3, State, Tuple2> mappingFunc = new Function3, State, Tuple2>() { - @Override public Tuple2 call(String word, Optional one, State state) { - int sum = one.or(0) + (state.exists() ? state.get() : 0); - Tuple2 output = new Tuple2(word, sum); + int sum = one.orElse(0) + (state.exists() ? state.get() : 0); + Tuple2 output = new Tuple2<>(word, sum); state.update(sum); return output; } diff --git a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java index 14975265ab2ce..27d494ce355f7 100644 --- a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java +++ b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java @@ -24,7 +24,6 @@ import scala.Tuple2; import com.google.common.collect.Iterables; -import com.google.common.base.Optional; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; @@ -38,6 +37,7 @@ 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.Optional; import org.apache.spark.api.java.function.*; import org.apache.spark.util.Utils; diff --git a/extras/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java index e8a0dfc0f0a5f..604d818ef1947 100644 --- a/extras/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java +++ b/extras/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java @@ -22,7 +22,6 @@ import scala.Tuple2; -import com.google.common.base.Optional; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import org.junit.Assert; diff --git a/network/common/pom.xml b/network/common/pom.xml index 32c34c63a45c5..92ca0046d4f53 100644 --- a/network/common/pom.xml +++ b/network/common/pom.xml @@ -52,15 +52,9 @@ com.google.code.findbugs jsr305 - com.google.guava guava - compile diff --git a/pom.xml b/pom.xml index e414a8bfe6ce5..9c975a45f8d23 100644 --- a/pom.xml +++ b/pom.xml @@ -2251,17 +2251,6 @@ com.google.common org.spark-project.guava - - - com/google/common/base/Absent* - com/google/common/base/Function - com/google/common/base/Optional* - com/google/common/base/Present* - com/google/common/base/Supplier - diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 40559a0910ce8..0d5f938d9ef5c 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -57,7 +57,16 @@ object MimaExcludes { ) ++ Seq( ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.SparkContext.emptyRDD"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.broadcast.HttpBroadcastFactory") - ) ++ + ) ++ + Seq( + // SPARK-4819 replace Guava Optional + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.JavaSparkContext.getCheckpointDir"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.JavaSparkContext.getSparkHome"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.getCheckpointFile"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitioner"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.getCheckpointFile"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitioner") + ) ++ Seq( // SPARK-12481 Remove Hadoop 1.x ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.mapred.SparkHadoopMapRedUtil"), diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StateSpec.scala b/streaming/src/main/scala/org/apache/spark/streaming/StateSpec.scala index 0b094558dfd59..f1114c1e5ac6a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StateSpec.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StateSpec.scala @@ -17,11 +17,9 @@ package org.apache.spark.streaming -import com.google.common.base.Optional - import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.annotation.Experimental -import org.apache.spark.api.java.{JavaPairRDD, JavaUtils} +import org.apache.spark.api.java.{JavaPairRDD, JavaUtils, Optional} import org.apache.spark.api.java.function.{Function3 => JFunction3, Function4 => JFunction4} import org.apache.spark.rdd.RDD import org.apache.spark.util.ClosureCleaner @@ -200,7 +198,11 @@ object StateSpec { StateSpec[KeyType, ValueType, StateType, MappedType] = { val wrappedFunc = (time: Time, k: KeyType, v: Option[ValueType], s: State[StateType]) => { val t = mappingFunction.call(time, k, JavaUtils.optionToOptional(v), s) - Option(t.orNull) + if (t.isPresent) { + Some(t.get) + } else { + None + } } StateSpec.function(wrappedFunc) } @@ -220,7 +222,7 @@ object StateSpec { mappingFunction: JFunction3[KeyType, Optional[ValueType], State[StateType], MappedType]): StateSpec[KeyType, ValueType, StateType, MappedType] = { val wrappedFunc = (k: KeyType, v: Option[ValueType], s: State[StateType]) => { - mappingFunction.call(k, Optional.fromNullable(v.get), s) + mappingFunction.call(k, Optional.ofNullable(v.get), s) } StateSpec.function(wrappedFunc) } 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 af0d84b33224f..d718f1d6fc43e 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 @@ -25,14 +25,13 @@ import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.ClassTag -import com.google.common.base.Optional import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.{JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.spark.Partitioner import org.apache.spark.annotation.Experimental -import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaUtils} +import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaUtils, Optional} import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index ddc56fc869ae1..4dbcef293487c 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -33,7 +33,6 @@ import org.junit.Assert; import org.junit.Test; -import com.google.common.base.Optional; import com.google.common.io.Files; import com.google.common.collect.Sets; @@ -43,6 +42,7 @@ 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.Optional; import org.apache.spark.api.java.function.*; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.api.java.*; diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java index 20e2a1c3d5c31..9b7701003d8d0 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java @@ -26,7 +26,6 @@ import scala.Tuple2; -import com.google.common.base.Optional; import com.google.common.collect.Sets; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.Function; @@ -38,6 +37,7 @@ import org.apache.spark.HashPartitioner; import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.Optional; import org.apache.spark.api.java.function.Function3; import org.apache.spark.api.java.function.Function4; import org.apache.spark.streaming.api.java.JavaPairDStream; @@ -139,7 +139,7 @@ public void testBasicFunction() { new Function3, State, Integer>() { @Override public Integer call(String key, Optional value, State state) { - int sum = value.or(0) + (state.exists() ? state.get() : 0); + int sum = value.orElse(0) + (state.exists() ? state.get() : 0); state.update(sum); return sum; } diff --git a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala index 6fb7184e877ee..ccd8fd3969f61 100644 --- a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala +++ b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala @@ -161,7 +161,7 @@ object JavaAPICompletenessChecker { } case "scala.Option" => { if (isReturnType) { - ParameterizedType("com.google.common.base.Optional", parameters.map(applySubs)) + ParameterizedType("org.apache.spark.api.java.Optional", parameters.map(applySubs)) } else { applySubs(parameters(0)) } From d9447cac747823e71b676c08c75f4aab34de12a2 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 8 Jan 2016 14:08:13 -0800 Subject: [PATCH 044/110] [SPARK-12593][SQL] Converts resolved logical plan back to SQL This PR tries to enable Spark SQL to convert resolved logical plans back to SQL query strings. For now, the major use case is to canonicalize Spark SQL native view support. The major entry point is `SQLBuilder.toSQL`, which returns an `Option[String]` if the logical plan is recognized. The current version is still in WIP status, and is quite limited. Known limitations include: 1. The logical plan must be analyzed but not optimized The optimizer erases `Subquery` operators, which contain necessary scope information for SQL generation. Future versions should be able to recover erased scope information by inserting subqueries when necessary. 1. The logical plan must be created using HiveQL query string Query plans generated by composing arbitrary DataFrame API combinations are not supported yet. Operators within these query plans need to be rearranged into a canonical form that is more suitable for direct SQL generation. For example, the following query plan ``` Filter (a#1 < 10) +- MetastoreRelation default, src, None ``` need to be canonicalized into the following form before SQL generation: ``` Project [a#1, b#2, c#3] +- Filter (a#1 < 10) +- MetastoreRelation default, src, None ``` Otherwise, the SQL generation process will have to handle a large number of special cases. 1. Only a fraction of expressions and basic logical plan operators are supported in this PR Currently, 95.7% (1720 out of 1798) query plans in `HiveCompatibilitySuite` can be successfully converted to SQL query strings. Known unsupported components are: - Expressions - Part of math expressions - Part of string expressions (buggy?) - Null expressions - Calendar interval literal - Part of date time expressions - Complex type creators - Special `NOT` expressions, e.g. `NOT LIKE` and `NOT IN` - Logical plan operators/patterns - Cube, rollup, and grouping set - Script transformation - Generator - Distinct aggregation patterns that fit `DistinctAggregationRewriter` analysis rule - Window functions Support for window functions, generators, and cubes etc. will be added in follow-up PRs. This PR leverages `HiveCompatibilitySuite` for testing SQL generation in a "round-trip" manner: * For all select queries, we try to convert it back to SQL * If the query plan is convertible, we parse the generated SQL into a new logical plan * Run the new logical plan instead of the original one If the query plan is inconvertible, the test case simply falls back to the original logic. TODO - [x] Fix failed test cases - [x] Support for more basic expressions and logical plan operators (e.g. distinct aggregation etc.) - [x] Comments and documentation Author: Cheng Lian Closes #10541 from liancheng/sql-generation. --- .../sql/catalyst/parser/SparkSqlParser.g | 48 ++-- .../sql/catalyst/analysis/Analyzer.scala | 20 +- .../spark/sql/catalyst/analysis/Catalog.scala | 4 +- .../spark/sql/catalyst/expressions/Cast.scala | 8 + .../sql/catalyst/expressions/Expression.scala | 23 +- .../catalyst/expressions/InputFileName.scala | 1 + .../MonotonicallyIncreasingID.scala | 4 + .../sql/catalyst/expressions/SortOrder.scala | 14 +- .../expressions/aggregate/interfaces.scala | 14 +- .../sql/catalyst/expressions/arithmetic.scala | 8 + .../expressions/complexTypeExtractors.scala | 2 + .../expressions/conditionalExpressions.scala | 41 ++- .../expressions/datetimeExpressions.scala | 22 ++ .../expressions/decimalExpressions.scala | 3 + .../sql/catalyst/expressions/literals.scala | 37 ++- .../expressions/mathExpressions.scala | 2 + .../spark/sql/catalyst/expressions/misc.scala | 4 + .../expressions/namedExpressions.scala | 12 + .../expressions/nullExpressions.scala | 6 + .../sql/catalyst/expressions/predicates.scala | 19 ++ .../expressions/randomExpressions.scala | 3 + .../expressions/regexpExpressions.scala | 2 + .../expressions/stringExpressions.scala | 28 +- .../sql/catalyst/optimizer/Optimizer.scala | 52 ++++ .../spark/sql/catalyst/plans/joinTypes.scala | 24 +- .../plans/logical/basicOperators.scala | 1 + .../sql/catalyst/rules/RuleExecutor.scala | 2 +- .../spark/sql/catalyst/util/package.scala | 14 + .../apache/spark/sql/types/ArrayType.scala | 2 + .../org/apache/spark/sql/types/DataType.scala | 2 + .../org/apache/spark/sql/types/MapType.scala | 2 + .../apache/spark/sql/types/StructType.scala | 5 + .../spark/sql/types/UserDefinedType.scala | 2 + .../sql/catalyst/analysis/AnalysisSuite.scala | 38 --- .../optimizer/ComputeCurrentTimeSuite.scala | 68 +++++ .../optimizer/FilterPushdownSuite.scala | 6 +- .../datasources/parquet/ParquetRelation.scala | 16 +- .../execution/HiveCompatibilitySuite.scala | 12 +- .../HiveWindowFunctionQuerySuite.scala | 1 + .../org/apache/spark/sql/hive/HiveQl.scala | 3 +- .../apache/spark/sql/hive/SQLBuilder.scala | 244 ++++++++++++++++++ .../org/apache/spark/sql/hive/hiveUDFs.scala | 48 ++-- .../sql/hive/ExpressionSQLBuilderSuite.scala | 75 ++++++ .../sql/hive/LogicalPlanToSQLSuite.scala | 146 +++++++++++ .../spark/sql/hive/SQLBuilderTest.scala | 74 ++++++ .../hive/execution/HiveComparisonTest.scala | 70 ++++- .../sql/hive/execution/HiveQuerySuite.scala | 1 + 47 files changed, 1087 insertions(+), 146 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ComputeCurrentTimeSuite.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/ExpressionSQLBuilderSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.scala diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g index b04bb677774c5..2c13d3056f468 100644 --- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g @@ -1,9 +1,9 @@ /** - Licensed to the Apache Software Foundation (ASF) under one or more - contributor license agreements. See the NOTICE file distributed with + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 - (the "License"); you may not use this file except in compliance with + (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 @@ -582,7 +582,7 @@ import java.util.HashMap; return header; } - + @Override public String getErrorMessage(RecognitionException e, String[] tokenNames) { String msg = null; @@ -619,7 +619,7 @@ import java.util.HashMap; } return msg; } - + public void pushMsg(String msg, RecognizerSharedState state) { // ANTLR generated code does not wrap the @init code wit this backtracking check, // even if the matching @after has it. If we have parser rules with that are doing @@ -639,7 +639,7 @@ import java.util.HashMap; // counter to generate unique union aliases private int aliasCounter; private String generateUnionAlias() { - return "_u" + (++aliasCounter); + return "u_" + (++aliasCounter); } private char [] excludedCharForColumnName = {'.', ':'}; private boolean containExcludedCharForCreateTableColumnName(String input) { @@ -1235,7 +1235,7 @@ alterTblPartitionStatementSuffixSkewedLocation : KW_SET KW_SKEWED KW_LOCATION skewedLocations -> ^(TOK_ALTERTABLE_SKEWED_LOCATION skewedLocations) ; - + skewedLocations @init { pushMsg("skewed locations", state); } @after { popMsg(state); } @@ -1264,7 +1264,7 @@ alterStatementSuffixLocation -> ^(TOK_ALTERTABLE_LOCATION $newLoc) ; - + alterStatementSuffixSkewedby @init {pushMsg("alter skewed by statement", state);} @after{popMsg(state);} @@ -1336,10 +1336,10 @@ tabTypeExpr (identifier (DOT^ ( (KW_ELEM_TYPE) => KW_ELEM_TYPE - | + | (KW_KEY_TYPE) => KW_KEY_TYPE - | - (KW_VALUE_TYPE) => KW_VALUE_TYPE + | + (KW_VALUE_TYPE) => KW_VALUE_TYPE | identifier ))* )? @@ -1376,7 +1376,7 @@ descStatement analyzeStatement @init { pushMsg("analyze statement", state); } @after { popMsg(state); } - : KW_ANALYZE KW_TABLE (parttype=tableOrPartition) KW_COMPUTE KW_STATISTICS ((noscan=KW_NOSCAN) | (partialscan=KW_PARTIALSCAN) + : KW_ANALYZE KW_TABLE (parttype=tableOrPartition) KW_COMPUTE KW_STATISTICS ((noscan=KW_NOSCAN) | (partialscan=KW_PARTIALSCAN) | (KW_FOR KW_COLUMNS (statsColumnName=columnNameList)?))? -> ^(TOK_ANALYZE $parttype $noscan? $partialscan? KW_COLUMNS? $statsColumnName?) ; @@ -1389,7 +1389,7 @@ showStatement | KW_SHOW KW_COLUMNS (KW_FROM|KW_IN) tableName ((KW_FROM|KW_IN) db_name=identifier)? -> ^(TOK_SHOWCOLUMNS tableName $db_name?) | KW_SHOW KW_FUNCTIONS (KW_LIKE showFunctionIdentifier|showFunctionIdentifier)? -> ^(TOK_SHOWFUNCTIONS KW_LIKE? showFunctionIdentifier?) - | KW_SHOW KW_PARTITIONS tabName=tableName partitionSpec? -> ^(TOK_SHOWPARTITIONS $tabName partitionSpec?) + | KW_SHOW KW_PARTITIONS tabName=tableName partitionSpec? -> ^(TOK_SHOWPARTITIONS $tabName partitionSpec?) | KW_SHOW KW_CREATE ( (KW_DATABASE|KW_SCHEMA) => (KW_DATABASE|KW_SCHEMA) db_name=identifier -> ^(TOK_SHOW_CREATEDATABASE $db_name) | @@ -1398,7 +1398,7 @@ showStatement | KW_SHOW KW_TABLE KW_EXTENDED ((KW_FROM|KW_IN) db_name=identifier)? KW_LIKE showStmtIdentifier partitionSpec? -> ^(TOK_SHOW_TABLESTATUS showStmtIdentifier $db_name? partitionSpec?) | KW_SHOW KW_TBLPROPERTIES tableName (LPAREN prptyName=StringLiteral RPAREN)? -> ^(TOK_SHOW_TBLPROPERTIES tableName $prptyName?) - | KW_SHOW KW_LOCKS + | KW_SHOW KW_LOCKS ( (KW_DATABASE|KW_SCHEMA) => (KW_DATABASE|KW_SCHEMA) (dbName=Identifier) (isExtended=KW_EXTENDED)? -> ^(TOK_SHOWDBLOCKS $dbName $isExtended?) | @@ -1511,7 +1511,7 @@ showCurrentRole setRole @init {pushMsg("set role", state);} @after {popMsg(state);} - : KW_SET KW_ROLE + : KW_SET KW_ROLE ( (KW_ALL) => (all=KW_ALL) -> ^(TOK_SHOW_SET_ROLE Identifier[$all.text]) | @@ -1966,7 +1966,7 @@ columnNameOrderList skewedValueElement @init { pushMsg("skewed value element", state); } @after { popMsg(state); } - : + : skewedColumnValues | skewedColumnValuePairList ; @@ -1980,8 +1980,8 @@ skewedColumnValuePairList skewedColumnValuePair @init { pushMsg("column value pair", state); } @after { popMsg(state); } - : - LPAREN colValues=skewedColumnValues RPAREN + : + LPAREN colValues=skewedColumnValues RPAREN -> ^(TOK_TABCOLVALUES $colValues) ; @@ -2001,11 +2001,11 @@ skewedColumnValue skewedValueLocationElement @init { pushMsg("skewed value location element", state); } @after { popMsg(state); } - : + : skewedColumnValue | skewedColumnValuePair ; - + columnNameOrder @init { pushMsg("column name order", state); } @after { popMsg(state); } @@ -2118,7 +2118,7 @@ unionType @after { popMsg(state); } : KW_UNIONTYPE LESSTHAN colTypeList GREATERTHAN -> ^(TOK_UNIONTYPE colTypeList) ; - + setOperator @init { pushMsg("set operator", state); } @after { popMsg(state); } @@ -2172,7 +2172,7 @@ fromStatement[boolean topLevel] {adaptor.create(Identifier, generateUnionAlias())} ) ) - ^(TOK_INSERT + ^(TOK_INSERT ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE)) ^(TOK_SELECT ^(TOK_SELEXPR TOK_ALLCOLREF)) ) @@ -2414,8 +2414,8 @@ setColumnsClause KW_SET columnAssignmentClause (COMMA columnAssignmentClause)* -> ^(TOK_SET_COLUMNS_CLAUSE columnAssignmentClause* ) ; -/* - UPDATE +/* + UPDATE
SET col1 = val1, col2 = val2... WHERE ... */ updateStatement 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 e362b55d80cd1..8a33af8207350 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 @@ -86,8 +86,7 @@ class Analyzer( HiveTypeCoercion.typeCoercionRules ++ extendedResolutionRules : _*), Batch("Nondeterministic", Once, - PullOutNondeterministic, - ComputeCurrentTime), + PullOutNondeterministic), Batch("UDF", Once, HandleNullInputsForUDF), Batch("Cleanup", fixedPoint, @@ -1229,23 +1228,6 @@ object CleanupAliases extends Rule[LogicalPlan] { } } -/** - * Computes the current date and time to make sure we return the same result in a single query. - */ -object ComputeCurrentTime extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = { - val dateExpr = CurrentDate() - val timeExpr = CurrentTimestamp() - val currentDate = Literal.create(dateExpr.eval(EmptyRow), dateExpr.dataType) - val currentTime = Literal.create(timeExpr.eval(EmptyRow), timeExpr.dataType) - - plan transformAllExpressions { - case CurrentDate() => currentDate - case CurrentTimestamp() => currentTime - } - } -} - /** * Replace the `UpCast` expression by `Cast`, and throw exceptions if the cast may truncate. */ 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 e8b2fcf819bf6..a8f89ce6de457 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 @@ -110,7 +110,9 @@ class SimpleCatalog(val conf: CatalystConf) extends Catalog { // If an alias was specified by the lookup, wrap the plan in a subquery so that attributes are // properly qualified with this alias. - alias.map(a => Subquery(a, tableWithQualifiers)).getOrElse(tableWithQualifiers) + alias + .map(a => Subquery(a, tableWithQualifiers)) + .getOrElse(tableWithQualifiers) } override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { 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 d82d3edae4e38..6f199cfc5d8cd 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 @@ -931,6 +931,14 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { $evPrim = $result.copy(); """ } + + override def sql: String = dataType match { + // HiveQL doesn't allow casting to complex types. For logical plans translated from HiveQL, this + // type of casting can only be introduced by the analyzer, and can be omitted when converting + // back to SQL query string. + case _: ArrayType | _: MapType | _: StructType => child.sql + case _ => s"CAST(${child.sql} AS ${dataType.sql})" + } } /** 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 6a9c12127d367..d6219514b752b 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 @@ -18,9 +18,10 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, TypeCheckResult, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.catalyst.util.sequenceOption import org.apache.spark.sql.types._ //////////////////////////////////////////////////////////////////////////////////////////////////// @@ -223,6 +224,15 @@ abstract class Expression extends TreeNode[Expression] { protected def toCommentSafeString: String = this.toString .replace("*/", "\\*\\/") .replace("\\u", "\\\\u") + + /** + * Returns SQL representation of this expression. For expressions that don't have a SQL + * representation (e.g. `ScalaUDF`), this method should throw an `UnsupportedOperationException`. + */ + @throws[UnsupportedOperationException](cause = "Expression doesn't have a SQL representation") + def sql: String = throw new UnsupportedOperationException( + s"Cannot map expression $this to its SQL representation" + ) } @@ -356,6 +366,8 @@ abstract class UnaryExpression extends Expression { """ } } + + override def sql: String = s"($prettyName(${child.sql}))" } @@ -456,6 +468,8 @@ abstract class BinaryExpression extends Expression { """ } } + + override def sql: String = s"$prettyName(${left.sql}, ${right.sql})" } @@ -492,6 +506,8 @@ abstract class BinaryOperator extends BinaryExpression with ExpectsInputTypes { TypeCheckResult.TypeCheckSuccess } } + + override def sql: String = s"(${left.sql} $symbol ${right.sql})" } @@ -593,4 +609,9 @@ abstract class TernaryExpression extends Expression { """ } } + + override def sql: String = { + val childrenSQL = children.map(_.sql).mkString(", ") + s"$prettyName($childrenSQL)" + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala index f33833c3918df..827dce8af100e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala @@ -49,4 +49,5 @@ case class InputFileName() extends LeafExpression with Nondeterministic { "org.apache.spark.rdd.SqlNewHadoopRDDState.getInputFileName();" } + override def sql: String = prettyName } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala index d0b78e15d99d1..94f8801dec369 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala @@ -78,4 +78,8 @@ private[sql] case class MonotonicallyIncreasingID() extends LeafExpression with $countTerm++; """ } + + override def prettyName: String = "monotonically_increasing_id" + + override def sql: String = s"$prettyName()" } 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 3add722da7816..1cb1b9da3049b 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 @@ -24,9 +24,17 @@ import org.apache.spark.sql.types._ import org.apache.spark.util.collection.unsafe.sort.PrefixComparators.BinaryPrefixComparator import org.apache.spark.util.collection.unsafe.sort.PrefixComparators.DoublePrefixComparator -abstract sealed class SortDirection -case object Ascending extends SortDirection -case object Descending extends SortDirection +abstract sealed class SortDirection { + def sql: String +} + +case object Ascending extends SortDirection { + override def sql: String = "ASC" +} + +case object Descending extends SortDirection { + override def sql: String = "DESC" +} /** * An expression that can be used to sort a tuple. This class extends expression primarily so that diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index b47f32d1768b9..ddd99c51ab0c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.catalyst.expressions.aggregate import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, CodegenFallback, GeneratedExpressionCode} +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.util.sequenceOption import org.apache.spark.sql.types._ /** The mode of an [[AggregateFunction]]. */ @@ -93,11 +94,13 @@ private[sql] case class AggregateExpression( override def prettyString: String = aggregateFunction.prettyString - override def toString: String = s"(${aggregateFunction},mode=$mode,isDistinct=$isDistinct)" + override def toString: String = s"($aggregateFunction,mode=$mode,isDistinct=$isDistinct)" + + override def sql: String = aggregateFunction.sql(isDistinct) } /** - * AggregateFunction2 is the superclass of two aggregation function interfaces: + * AggregateFunction is the superclass of two aggregation function interfaces: * * - [[ImperativeAggregate]] is for aggregation functions that are specified in terms of * initialize(), update(), and merge() functions that operate on Row-based aggregation buffers. @@ -163,6 +166,11 @@ sealed abstract class AggregateFunction extends Expression with ImplicitCastInpu def toAggregateExpression(isDistinct: Boolean): AggregateExpression = { AggregateExpression(aggregateFunction = this, mode = Complete, isDistinct = isDistinct) } + + def sql(isDistinct: Boolean): String = { + val distinct = if (isDistinct) "DISTINCT " else " " + s"$prettyName($distinct${children.map(_.sql).mkString(", ")})" + } } /** 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 61a17fd7db0fe..7bd851c059d0e 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 @@ -54,6 +54,8 @@ case class UnaryMinus(child: Expression) extends UnaryExpression with ExpectsInp numeric.negate(input) } } + + override def sql: String = s"(-${child.sql})" } case class UnaryPositive(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -67,6 +69,8 @@ case class UnaryPositive(child: Expression) extends UnaryExpression with Expects defineCodeGen(ctx, ev, c => c) protected override def nullSafeEval(input: Any): Any = input + + override def sql: String = s"(+${child.sql})" } /** @@ -91,6 +95,8 @@ case class Abs(child: Expression) extends UnaryExpression with ExpectsInputTypes } protected override def nullSafeEval(input: Any): Any = numeric.abs(input) + + override def sql: String = s"$prettyName(${child.sql})" } abstract class BinaryArithmetic extends BinaryOperator { @@ -513,4 +519,6 @@ case class Pmod(left: Expression, right: Expression) extends BinaryArithmetic { val r = a % n if (r.compare(Decimal.ZERO) < 0) {(r + n) % n} else r } + + override def sql: String = s"$prettyName(${left.sql}, ${right.sql})" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala index 9c73239f67ff2..5bd97cc7467ab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala @@ -130,6 +130,8 @@ case class GetStructField(child: Expression, ordinal: Int, name: Option[String] } }) } + + override def sql: String = child.sql + s".`${childSchema(ordinal).name}`" } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index f79c8676fb58c..19da849d2bec9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.util.TypeUtils +import org.apache.spark.sql.catalyst.util.{sequenceOption, TypeUtils} import org.apache.spark.sql.types._ @@ -74,6 +74,8 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi } override def toString: String = s"if ($predicate) $trueValue else $falseValue" + + override def sql: String = s"(IF(${predicate.sql}, ${trueValue.sql}, ${falseValue.sql}))" } trait CaseWhenLike extends Expression { @@ -110,7 +112,7 @@ trait CaseWhenLike extends Expression { override def nullable: Boolean = { // If no value is nullable and no elseValue is provided, the whole statement defaults to null. - thenList.exists(_.nullable) || (elseValue.map(_.nullable).getOrElse(true)) + thenList.exists(_.nullable) || elseValue.map(_.nullable).getOrElse(true) } } @@ -206,6 +208,23 @@ case class CaseWhen(branches: Seq[Expression]) extends CaseWhenLike { case Seq(elseValue) => s" ELSE $elseValue" }.mkString } + + override def sql: String = { + val branchesSQL = branches.map(_.sql) + val (cases, maybeElse) = if (branches.length % 2 == 0) { + (branchesSQL, None) + } else { + (branchesSQL.init, Some(branchesSQL.last)) + } + + val head = s"CASE " + val tail = maybeElse.map(e => s" ELSE $e").getOrElse("") + " END" + val body = cases.grouped(2).map { + case Seq(whenExpr, thenExpr) => s"WHEN $whenExpr THEN $thenExpr" + }.mkString(" ") + + head + body + tail + } } // scalastyle:off @@ -310,6 +329,24 @@ case class CaseKeyWhen(key: Expression, branches: Seq[Expression]) extends CaseW case Seq(elseValue) => s" ELSE $elseValue" }.mkString } + + override def sql: String = { + val keySQL = key.sql + val branchesSQL = branches.map(_.sql) + val (cases, maybeElse) = if (branches.length % 2 == 0) { + (branchesSQL, None) + } else { + (branchesSQL.init, Some(branchesSQL.last)) + } + + val head = s"CASE $keySQL " + val tail = maybeElse.map(e => s" ELSE $e").getOrElse("") + " END" + val body = cases.grouped(2).map { + case Seq(whenExpr, thenExpr) => s"WHEN $whenExpr THEN $thenExpr" + }.mkString(" ") + + head + body + tail + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 3d65946a1bc65..17f1df06f2fad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -44,6 +44,8 @@ case class CurrentDate() extends LeafExpression with CodegenFallback { override def eval(input: InternalRow): Any = { DateTimeUtils.millisToDays(System.currentTimeMillis()) } + + override def prettyName: String = "current_date" } /** @@ -61,6 +63,8 @@ case class CurrentTimestamp() extends LeafExpression with CodegenFallback { override def eval(input: InternalRow): Any = { System.currentTimeMillis() * 1000L } + + override def prettyName: String = "current_timestamp" } /** @@ -85,6 +89,8 @@ case class DateAdd(startDate: Expression, days: Expression) s"""${ev.value} = $sd + $d;""" }) } + + override def prettyName: String = "date_add" } /** @@ -108,6 +114,8 @@ case class DateSub(startDate: Expression, days: Expression) s"""${ev.value} = $sd - $d;""" }) } + + override def prettyName: String = "date_sub" } case class Hour(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { @@ -309,6 +317,8 @@ case class ToUnixTimestamp(timeExp: Expression, format: Expression) extends Unix def this(time: Expression) = { this(time, Literal("yyyy-MM-dd HH:mm:ss")) } + + override def prettyName: String = "to_unix_timestamp" } /** @@ -332,6 +342,8 @@ case class UnixTimestamp(timeExp: Expression, format: Expression) extends UnixTi def this() = { this(CurrentTimestamp()) } + + override def prettyName: String = "unix_timestamp" } abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { @@ -437,6 +449,8 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { """ } } + + override def prettyName: String = "unix_time" } /** @@ -451,6 +465,8 @@ case class FromUnixTime(sec: Expression, format: Expression) override def left: Expression = sec override def right: Expression = format + override def prettyName: String = "from_unixtime" + def this(unix: Expression) = { this(unix, Literal("yyyy-MM-dd HH:mm:ss")) } @@ -733,6 +749,8 @@ case class AddMonths(startDate: Expression, numMonths: Expression) s"""$dtu.dateAddMonths($sd, $m)""" }) } + + override def prettyName: String = "add_months" } /** @@ -758,6 +776,8 @@ case class MonthsBetween(date1: Expression, date2: Expression) s"""$dtu.monthsBetween($l, $r)""" }) } + + override def prettyName: String = "months_between" } /** @@ -823,6 +843,8 @@ case class ToDate(child: Expression) extends UnaryExpression with ImplicitCastIn override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { defineCodeGen(ctx, ev, d => d) } + + override def prettyName: String = "to_date" } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala index c54bcdd774021..5f8b544edb511 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala @@ -73,6 +73,7 @@ case class PromotePrecision(child: Expression) extends UnaryExpression { override def gen(ctx: CodeGenContext): GeneratedExpressionCode = child.gen(ctx) override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = "" override def prettyName: String = "promote_precision" + override def sql: String = child.sql } /** @@ -107,4 +108,6 @@ case class CheckOverflow(child: Expression, dataType: DecimalType) extends Unary } override def toString: String = s"CheckOverflow($child, $dataType)" + + override def sql: String = child.sql } 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 672cc9c45e0af..0eb915fdc1691 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 @@ -21,9 +21,9 @@ import java.sql.{Date, Timestamp} import org.json4s.JsonAST._ -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types._ @@ -214,6 +214,41 @@ case class Literal protected (value: Any, dataType: DataType) } } } + + override def sql: String = (value, dataType) match { + case (_, NullType | _: ArrayType | _: MapType | _: StructType) if value == null => + "NULL" + + case _ if value == null => + s"CAST(NULL AS ${dataType.sql})" + + case (v: UTF8String, StringType) => + // Escapes all backslashes and double quotes. + "\"" + v.toString.replace("\\", "\\\\").replace("\"", "\\\"") + "\"" + + case (v: Byte, ByteType) => + s"CAST($v AS ${ByteType.simpleString.toUpperCase})" + + case (v: Short, ShortType) => + s"CAST($v AS ${ShortType.simpleString.toUpperCase})" + + case (v: Long, LongType) => + s"CAST($v AS ${LongType.simpleString.toUpperCase})" + + case (v: Float, FloatType) => + s"CAST($v AS ${FloatType.simpleString.toUpperCase})" + + case (v: Decimal, DecimalType.Fixed(precision, scale)) => + s"CAST($v AS ${DecimalType.simpleString.toUpperCase}($precision, $scale))" + + case (v: Int, DateType) => + s"DATE '${DateTimeUtils.toJavaDate(v)}'" + + case (v: Long, TimestampType) => + s"TIMESTAMP('${DateTimeUtils.toJavaTimestamp(v)}')" + + case _ => value.toString + } } // TODO: Specialize diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index 002f5929cc26b..66d8631a846ab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -70,6 +70,8 @@ abstract class UnaryMathExpression(val f: Double => Double, name: String) override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { defineCodeGen(ctx, ev, c => s"java.lang.Math.${funcName}($c)") } + + override def sql: String = s"$name(${child.sql})" } abstract class UnaryLogExpression(f: Double => Double, name: String) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index fd95b124b2455..cc406a39f0408 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -220,4 +220,8 @@ case class Murmur3Hash(children: Seq[Expression], seed: Int) extends Expression final int ${ev.value} = ${unsafeRow.value}.hashCode($seed); """ } + + override def prettyName: String = "hash" + + override def sql: String = s"$prettyName(${children.map(_.sql).mkString(", ")}, $seed)" } 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 eefd9c7482553..eee708cb02f9d 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 @@ -164,6 +164,12 @@ case class Alias(child: Expression, name: String)( explicitMetadata == a.explicitMetadata case _ => false } + + override def sql: String = { + val qualifiersString = + if (qualifiers.isEmpty) "" else qualifiers.map("`" + _ + "`").mkString("", ".", ".") + s"${child.sql} AS $qualifiersString`$name`" + } } /** @@ -271,6 +277,12 @@ case class AttributeReference( // Since the expression id is not in the first constructor it is missing from the default // tree string. override def simpleString: String = s"$name#${exprId.id}: ${dataType.simpleString}" + + override def sql: String = { + val qualifiersString = + if (qualifiers.isEmpty) "" else qualifiers.map("`" + _ + "`").mkString("", ".", ".") + s"$qualifiersString`$name`" + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala index df4747d4e6f7a..89aec2b20fd0c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala @@ -83,6 +83,8 @@ case class Coalesce(children: Seq[Expression]) extends Expression { """ }.mkString("\n") } + + override def sql: String = s"$prettyName(${children.map(_.sql).mkString(", ")})" } @@ -193,6 +195,8 @@ case class IsNull(child: Expression) extends UnaryExpression with Predicate { ev.value = eval.isNull eval.code } + + override def sql: String = s"(${child.sql} IS NULL)" } @@ -212,6 +216,8 @@ case class IsNotNull(child: Expression) extends UnaryExpression with Predicate { ev.value = s"(!(${eval.isNull}))" eval.code } + + override def sql: String = s"(${child.sql} IS NOT NULL)" } 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 304b438c84ba4..bca12a8d21023 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 @@ -101,6 +101,8 @@ case class Not(child: Expression) override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { defineCodeGen(ctx, ev, c => s"!($c)") } + + override def sql: String = s"(NOT ${child.sql})" } @@ -176,6 +178,13 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate } """ } + + override def sql: String = { + val childrenSQL = children.map(_.sql) + val valueSQL = childrenSQL.head + val listSQL = childrenSQL.tail.mkString(", ") + s"($valueSQL IN ($listSQL))" + } } /** @@ -226,6 +235,12 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with } """ } + + override def sql: String = { + val valueSQL = child.sql + val listSQL = hset.toSeq.map(Literal(_).sql).mkString(", ") + s"($valueSQL IN ($listSQL))" + } } case class And(left: Expression, right: Expression) extends BinaryOperator with Predicate { @@ -274,6 +289,8 @@ case class And(left: Expression, right: Expression) extends BinaryOperator with } """ } + + override def sql: String = s"(${left.sql} AND ${right.sql})" } @@ -323,6 +340,8 @@ case class Or(left: Expression, right: Expression) extends BinaryOperator with P } """ } + + override def sql: String = s"(${left.sql} OR ${right.sql})" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala index 8bde8cb9fe876..8de47e9ddc28d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala @@ -49,6 +49,9 @@ abstract class RDG extends LeafExpression with Nondeterministic { override def nullable: Boolean = false override def dataType: DataType = DoubleType + + // NOTE: Even if the user doesn't provide a seed, Spark SQL adds a default seed. + override def sql: String = s"$prettyName($seed)" } /** Generate a random column with i.i.d. uniformly distributed values in [0, 1). */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index adef6050c3565..db266639b8560 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -59,6 +59,8 @@ trait StringRegexExpression extends ImplicitCastInputTypes { matches(regex, input1.asInstanceOf[UTF8String].toString) } } + + override def sql: String = s"${left.sql} ${prettyName.toUpperCase} ${right.sql}" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index 50c8b9d59847e..931f752b4dc1a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -23,6 +23,7 @@ import java.util.{HashMap, Locale, Map => JMap} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.catalyst.util.sequenceOption import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{ByteArray, UTF8String} @@ -61,6 +62,8 @@ case class Concat(children: Seq[Expression]) extends Expression with ImplicitCas } """ } + + override def sql: String = s"$prettyName(${children.map(_.sql).mkString(", ")})" } @@ -153,6 +156,8 @@ case class ConcatWs(children: Seq[Expression]) """ } } + + override def sql: String = s"$prettyName(${children.map(_.sql).mkString(", ")})" } trait String2StringExpression extends ImplicitCastInputTypes { @@ -292,24 +297,24 @@ case class StringTranslate(srcExpr: Expression, matchingExpr: Expression, replac val termDict = ctx.freshName("dict") val classNameDict = classOf[JMap[Character, Character]].getCanonicalName - ctx.addMutableState("UTF8String", termLastMatching, s"${termLastMatching} = null;") - ctx.addMutableState("UTF8String", termLastReplace, s"${termLastReplace} = null;") - ctx.addMutableState(classNameDict, termDict, s"${termDict} = null;") + ctx.addMutableState("UTF8String", termLastMatching, s"$termLastMatching = null;") + ctx.addMutableState("UTF8String", termLastReplace, s"$termLastReplace = null;") + ctx.addMutableState(classNameDict, termDict, s"$termDict = null;") nullSafeCodeGen(ctx, ev, (src, matching, replace) => { val check = if (matchingExpr.foldable && replaceExpr.foldable) { - s"${termDict} == null" + s"$termDict == null" } else { - s"!${matching}.equals(${termLastMatching}) || !${replace}.equals(${termLastReplace})" + s"!$matching.equals($termLastMatching) || !$replace.equals($termLastReplace)" } s"""if ($check) { // Not all of them is literal or matching or replace value changed - ${termLastMatching} = ${matching}.clone(); - ${termLastReplace} = ${replace}.clone(); - ${termDict} = org.apache.spark.sql.catalyst.expressions.StringTranslate - .buildDict(${termLastMatching}, ${termLastReplace}); + $termLastMatching = $matching.clone(); + $termLastReplace = $replace.clone(); + $termDict = org.apache.spark.sql.catalyst.expressions.StringTranslate + .buildDict($termLastMatching, $termLastReplace); } - ${ev.value} = ${src}.translate(${termDict}); + ${ev.value} = $src.translate($termDict); """ }) } @@ -340,6 +345,8 @@ case class FindInSet(left: Expression, right: Expression) extends BinaryExpressi } override def dataType: DataType = IntegerType + + override def prettyName: String = "find_in_set" } /** @@ -832,7 +839,6 @@ case class Base64(child: Expression) extends UnaryExpression with ImplicitCastIn org.apache.commons.codec.binary.Base64.encodeBase64($child)); """}) } - } /** 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 0b1c74293bb8b..f8121a733a8d2 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 @@ -37,6 +37,8 @@ abstract class Optimizer extends RuleExecutor[LogicalPlan] { // SubQueries are only needed for analysis and can be removed before execution. Batch("Remove SubQueries", FixedPoint(100), EliminateSubQueries) :: + Batch("Compute Current Time", Once, + ComputeCurrentTime) :: Batch("Aggregate", FixedPoint(100), ReplaceDistinctWithAggregate, RemoveLiteralFromGroupExpressions) :: @@ -333,6 +335,39 @@ object ProjectCollapsing extends Rule[LogicalPlan] { ) Project(cleanedProjection, child) } + + // TODO Eliminate duplicate code + // This clause is identical to the one above except that the inner operator is an `Aggregate` + // rather than a `Project`. + case p @ Project(projectList1, agg @ Aggregate(_, projectList2, child)) => + // Create a map of Aliases to their values from the child projection. + // e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> Alias(a + b, c)). + val aliasMap = AttributeMap(projectList2.collect { + case a: Alias => (a.toAttribute, a) + }) + + // We only collapse these two Projects if their overlapped expressions are all + // deterministic. + val hasNondeterministic = projectList1.exists(_.collect { + case a: Attribute if aliasMap.contains(a) => aliasMap(a).child + }.exists(!_.deterministic)) + + if (hasNondeterministic) { + p + } else { + // Substitute any attributes that are produced by the child projection, so that we safely + // eliminate it. + // e.g., 'SELECT c + 1 FROM (SELECT a + b AS C ...' produces 'SELECT a + b + 1 ...' + // TODO: Fix TransformBase to avoid the cast below. + val substitutedProjection = projectList1.map(_.transform { + case a: Attribute => aliasMap.getOrElse(a, a) + }).asInstanceOf[Seq[NamedExpression]] + // collapse 2 projects may introduce unnecessary Aliases, trim them here. + val cleanedProjection = substitutedProjection.map(p => + CleanupAliases.trimNonTopLevelAliases(p).asInstanceOf[NamedExpression] + ) + agg.copy(aggregateExpressions = cleanedProjection) + } } } @@ -976,3 +1011,20 @@ object RemoveLiteralFromGroupExpressions extends Rule[LogicalPlan] { a.copy(groupingExpressions = newGrouping) } } + +/** + * Computes the current date and time to make sure we return the same result in a single query. + */ +object ComputeCurrentTime extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + val dateExpr = CurrentDate() + val timeExpr = CurrentTimestamp() + val currentDate = Literal.create(dateExpr.eval(EmptyRow), dateExpr.dataType) + val currentTime = Literal.create(timeExpr.eval(EmptyRow), timeExpr.dataType) + + plan transformAllExpressions { + case CurrentDate() => currentDate + case CurrentTimestamp() => currentTime + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala index 77dec7ca6e2b5..a5f6764aef7ce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala @@ -37,14 +37,26 @@ object JoinType { } } -sealed abstract class JoinType +sealed abstract class JoinType { + def sql: String +} -case object Inner extends JoinType +case object Inner extends JoinType { + override def sql: String = "INNER" +} -case object LeftOuter extends JoinType +case object LeftOuter extends JoinType { + override def sql: String = "LEFT OUTER" +} -case object RightOuter extends JoinType +case object RightOuter extends JoinType { + override def sql: String = "RIGHT OUTER" +} -case object FullOuter extends JoinType +case object FullOuter extends JoinType { + override def sql: String = "FULL OUTER" +} -case object LeftSemi extends JoinType +case object LeftSemi extends JoinType { + override def sql: String = "LEFT SEMI" +} 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 79759b5a37b34..64957db6b4013 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 @@ -423,6 +423,7 @@ case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { } case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { + override def output: Seq[Attribute] = child.output.map(_.withQualifiers(alias :: Nil)) } 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 62ea731ab5f38..9ebacb4680dc2 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 @@ -37,7 +37,7 @@ object RuleExecutor { val maxSize = map.keys.map(_.toString.length).max map.toSeq.sortBy(_._2).reverseMap { case (k, v) => s"${k.padTo(maxSize, " ").mkString} $v" - }.mkString("\n") + }.mkString("\n", "\n", "") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 71293475ca0f9..7a0d0de6328a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -130,6 +130,20 @@ package object util { ret } + /** + * Converts a `Seq` of `Option[T]` to an `Option` of `Seq[T]`. + */ + def sequenceOption[T](seq: Seq[Option[T]]): Option[Seq[T]] = seq match { + case xs if xs.isEmpty => + Option(Seq.empty[T]) + + case xs => + for { + head <- xs.head + tail <- sequenceOption(xs.tail) + } yield head +: tail + } + /* FIX ME implicit class debugLogging(a: Any) { def debugLogging() { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala index 6533622492d41..520e344361625 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala @@ -77,6 +77,8 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT override def simpleString: String = s"array<${elementType.simpleString}>" + override def sql: String = s"ARRAY<${elementType.sql}>" + override private[spark] def asNullable: ArrayType = ArrayType(elementType.asNullable, containsNull = true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala index 136a97e066df7..92cf8d4c46bda 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -65,6 +65,8 @@ abstract class DataType extends AbstractDataType { /** Readable string representation for the type with truncation */ private[sql] def simpleString(maxNumberFields: Int): String = simpleString + def sql: String = simpleString.toUpperCase + /** * Check if `this` and `other` are the same data type when ignoring nullability * (`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`). diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala index 00461e529ca0a..5474954af70e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala @@ -62,6 +62,8 @@ case class MapType( override def simpleString: String = s"map<${keyType.simpleString},${valueType.simpleString}>" + override def sql: String = s"MAP<${keyType.sql}, ${valueType.sql}>" + override private[spark] def asNullable: MapType = MapType(keyType.asNullable, valueType.asNullable, valueContainsNull = true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 34382bf124eb0..9b5c86a8984be 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -279,6 +279,11 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru s"struct<${fieldTypes.mkString(",")}>" } + override def sql: String = { + val fieldTypes = fields.map(f => s"`${f.name}`: ${f.dataType.sql}") + s"STRUCT<${fieldTypes.mkString(", ")}>" + } + private[sql] override def simpleString(maxNumberFields: Int): String = { val builder = new StringBuilder val fieldTypes = fields.take(maxNumberFields).map { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala index 4305903616bd9..d7a2c23be8a9a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala @@ -84,6 +84,8 @@ abstract class UserDefinedType[UserType] extends DataType with Serializable { override private[sql] def acceptsType(dataType: DataType) = this.getClass == dataType.getClass + + override def sql: String = sqlType.sql } /** 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 fa823e3021835..cf84855885a37 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 @@ -22,7 +22,6 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ class AnalysisSuite extends AnalysisTest { @@ -238,43 +237,6 @@ class AnalysisSuite extends AnalysisTest { checkAnalysis(plan, expected) } - test("analyzer should replace current_timestamp with literals") { - val in = Project(Seq(Alias(CurrentTimestamp(), "a")(), Alias(CurrentTimestamp(), "b")()), - LocalRelation()) - - val min = System.currentTimeMillis() * 1000 - val plan = in.analyze.asInstanceOf[Project] - val max = (System.currentTimeMillis() + 1) * 1000 - - val lits = new scala.collection.mutable.ArrayBuffer[Long] - plan.transformAllExpressions { case e: Literal => - lits += e.value.asInstanceOf[Long] - e - } - assert(lits.size == 2) - assert(lits(0) >= min && lits(0) <= max) - assert(lits(1) >= min && lits(1) <= max) - assert(lits(0) == lits(1)) - } - - test("analyzer should replace current_date with literals") { - val in = Project(Seq(Alias(CurrentDate(), "a")(), Alias(CurrentDate(), "b")()), LocalRelation()) - - val min = DateTimeUtils.millisToDays(System.currentTimeMillis()) - val plan = in.analyze.asInstanceOf[Project] - val max = DateTimeUtils.millisToDays(System.currentTimeMillis()) - - val lits = new scala.collection.mutable.ArrayBuffer[Int] - plan.transformAllExpressions { case e: Literal => - lits += e.value.asInstanceOf[Int] - e - } - assert(lits.size == 2) - assert(lits(0) >= min && lits(0) <= max) - assert(lits(1) >= min && lits(1) <= max) - assert(lits(0) == lits(1)) - } - test("SPARK-12102: Ignore nullablity when comparing two sides of case") { val relation = LocalRelation('a.struct('x.int), 'b.struct('x.int.withNullability(false))) val plan = relation.select(CaseWhen(Seq(Literal(true), 'a, 'b)).as("val")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ComputeCurrentTimeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ComputeCurrentTimeSuite.scala new file mode 100644 index 0000000000000..10ed4e46ddd1c --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ComputeCurrentTimeSuite.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.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{Alias, CurrentDate, CurrentTimestamp, Literal} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.catalyst.util.DateTimeUtils + +class ComputeCurrentTimeSuite extends PlanTest { + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Seq(Batch("ComputeCurrentTime", Once, ComputeCurrentTime)) + } + + test("analyzer should replace current_timestamp with literals") { + val in = Project(Seq(Alias(CurrentTimestamp(), "a")(), Alias(CurrentTimestamp(), "b")()), + LocalRelation()) + + val min = System.currentTimeMillis() * 1000 + val plan = Optimize.execute(in.analyze).asInstanceOf[Project] + val max = (System.currentTimeMillis() + 1) * 1000 + + val lits = new scala.collection.mutable.ArrayBuffer[Long] + plan.transformAllExpressions { case e: Literal => + lits += e.value.asInstanceOf[Long] + e + } + assert(lits.size == 2) + assert(lits(0) >= min && lits(0) <= max) + assert(lits(1) >= min && lits(1) <= max) + assert(lits(0) == lits(1)) + } + + test("analyzer should replace current_date with literals") { + val in = Project(Seq(Alias(CurrentDate(), "a")(), Alias(CurrentDate(), "b")()), LocalRelation()) + + val min = DateTimeUtils.millisToDays(System.currentTimeMillis()) + val plan = Optimize.execute(in.analyze).asInstanceOf[Project] + val max = DateTimeUtils.millisToDays(System.currentTimeMillis()) + + val lits = new scala.collection.mutable.ArrayBuffer[Int] + plan.transformAllExpressions { case e: Literal => + lits += e.value.asInstanceOf[Int] + e + } + assert(lits.size == 2) + assert(lits(0) >= min && lits(0) <= max) + assert(lits(1) >= min && lits(1) <= max) + assert(lits(0) == lits(1)) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index b998636909a7d..f9f3bd55aa578 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -75,8 +75,7 @@ class FilterPushdownSuite extends PlanTest { val correctAnswer = testRelation .select('a) - .groupBy('a)('a) - .select('a).analyze + .groupBy('a)('a).analyze comparePlans(optimized, correctAnswer) } @@ -91,8 +90,7 @@ class FilterPushdownSuite extends PlanTest { val correctAnswer = testRelation .select('a) - .groupBy('a)('a as 'c) - .select('c).analyze + .groupBy('a)('a as 'c).analyze comparePlans(optimized, correctAnswer) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index 4b375de05e9e3..ca8d010090401 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.execution.datasources.parquet import java.net.URI -import java.util.{List => JList} import java.util.logging.{Logger => JLogger} +import java.util.{List => JList} import scala.collection.JavaConverters._ import scala.collection.mutable @@ -32,24 +32,24 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.task.JobContextImpl -import org.apache.parquet.{Log => ApacheParquetLog} import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.hadoop._ import org.apache.parquet.hadoop.metadata.CompressionCodecName import org.apache.parquet.hadoop.util.ContextUtil import org.apache.parquet.schema.MessageType +import org.apache.parquet.{Log => ApacheParquetLog} import org.slf4j.bridge.SLF4JBridgeHandler -import org.apache.spark.{Logging, Partition => SparkPartition, SparkException} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{RDD, SqlNewHadoopPartition, SqlNewHadoopRDD} import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.catalyst.util.LegacyTypeStringParser +import org.apache.spark.sql.catalyst.{InternalRow, SqlParser, TableIdentifier} +import org.apache.spark.sql.execution.datasources.{PartitionSpec, _} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.{SerializableConfiguration, Utils} +import org.apache.spark.{Logging, Partition => SparkPartition, SparkException} private[sql] class DefaultSource extends BucketedHadoopFsRelationProvider with DataSourceRegister { @@ -147,6 +147,12 @@ private[sql] class ParquetRelation( .get(ParquetRelation.METASTORE_SCHEMA) .map(DataType.fromJson(_).asInstanceOf[StructType]) + // If this relation is converted from a Hive metastore table, this method returns the name of the + // original Hive metastore table. + private[sql] def metastoreTableName: Option[TableIdentifier] = { + parameters.get(ParquetRelation.METASTORE_TABLE_NAME).map(SqlParser.parseTableIdentifier) + } + private lazy val metadataCache: MetadataCache = { val meta = new MetadataCache meta.refresh() 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 bd1a52e5f3303..afd2f611580fc 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 @@ -41,9 +41,12 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { private val originalColumnBatchSize = TestHive.conf.columnBatchSize private val originalInMemoryPartitionPruning = TestHive.conf.inMemoryPartitionPruning - def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) + def testCases: Seq[(String, File)] = { + hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) + } override def beforeAll() { + super.beforeAll() TestHive.cacheTables = true // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) @@ -68,10 +71,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // For debugging dump some statistics about how much time was spent in various optimizer rules. logWarning(RuleExecutor.dumpTimeSpent()) + super.afterAll() } /** A list of tests deemed out of scope currently and thus completely disregarded. */ - override def blackList = Seq( + override def blackList: Seq[String] = Seq( // These tests use hooks that are not on the classpath and thus break all subsequent execution. "hook_order", "hook_context_cs", @@ -106,7 +110,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "alter_merge", "alter_concatenate_indexed_table", "protectmode2", - //"describe_table", + // "describe_table", "describe_comment_nonascii", "create_merge_compressed", @@ -323,7 +327,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { * The set of tests that are believed to be working in catalyst. Tests not on whiteList or * blacklist are implicitly marked as ignored. */ - override def whiteList = Seq( + override def whiteList: Seq[String] = Seq( "add_part_exist", "add_part_multiple", "add_partition_no_whitelist", diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala index 98bbdf0653c2a..bad3ca6da231f 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -104,6 +104,7 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte TimeZone.setDefault(originalTimeZone) Locale.setDefault(originalLocale) TestHive.reset() + super.afterAll() } ///////////////////////////////////////////////////////////////////////////// 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 bf3fe12d5c5d2..5b13dbe47370e 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 @@ -668,7 +668,8 @@ private[hive] object HiveQl extends SparkQl with Logging { Option(FunctionRegistry.getFunctionInfo(functionName.toLowerCase)).getOrElse( sys.error(s"Couldn't find function $functionName")) val functionClassName = functionInfo.getFunctionClass.getName - HiveGenericUDTF(new HiveFunctionWrapper(functionClassName), children.map(nodeToExpr)) + HiveGenericUDTF( + functionName, new HiveFunctionWrapper(functionClassName), children.map(nodeToExpr)) case other => super.nodeToGenerator(node) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala new file mode 100644 index 0000000000000..1c910051faccf --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala @@ -0,0 +1,244 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.concurrent.atomic.AtomicLong + +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, NamedExpression, SortOrder} +import org.apache.spark.sql.catalyst.optimizer.ProjectCollapsing +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor} +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation +import org.apache.spark.sql.{DataFrame, SQLContext} + +/** + * A builder class used to convert a resolved logical plan into a SQL query string. Note that this + * all resolved logical plan are convertible. They either don't have corresponding SQL + * representations (e.g. logical plans that operate on local Scala collections), or are simply not + * supported by this builder (yet). + */ +class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Logging { + def this(df: DataFrame) = this(df.queryExecution.analyzed, df.sqlContext) + + def toSQL: Option[String] = { + val canonicalizedPlan = Canonicalizer.execute(logicalPlan) + val maybeSQL = try { + toSQL(canonicalizedPlan) + } catch { case cause: UnsupportedOperationException => + logInfo(s"Failed to build SQL query string because: ${cause.getMessage}") + None + } + + if (maybeSQL.isDefined) { + logDebug( + s"""Built SQL query string successfully from given logical plan: + | + |# Original logical plan: + |${logicalPlan.treeString} + |# Canonicalized logical plan: + |${canonicalizedPlan.treeString} + |# Built SQL query string: + |${maybeSQL.get} + """.stripMargin) + } else { + logDebug( + s"""Failed to build SQL query string from given logical plan: + | + |# Original logical plan: + |${logicalPlan.treeString} + |# Canonicalized logical plan: + |${canonicalizedPlan.treeString} + """.stripMargin) + } + + maybeSQL + } + + private def projectToSQL( + projectList: Seq[NamedExpression], + child: LogicalPlan, + isDistinct: Boolean): Option[String] = { + for { + childSQL <- toSQL(child) + listSQL = projectList.map(_.sql).mkString(", ") + maybeFrom = child match { + case OneRowRelation => " " + case _ => " FROM " + } + distinct = if (isDistinct) " DISTINCT " else " " + } yield s"SELECT$distinct$listSQL$maybeFrom$childSQL" + } + + private def aggregateToSQL( + groupingExprs: Seq[Expression], + aggExprs: Seq[Expression], + child: LogicalPlan): Option[String] = { + val aggSQL = aggExprs.map(_.sql).mkString(", ") + val groupingSQL = groupingExprs.map(_.sql).mkString(", ") + val maybeGroupBy = if (groupingSQL.isEmpty) "" else " GROUP BY " + val maybeFrom = child match { + case OneRowRelation => " " + case _ => " FROM " + } + + toSQL(child).map { childSQL => + s"SELECT $aggSQL$maybeFrom$childSQL$maybeGroupBy$groupingSQL" + } + } + + private def toSQL(node: LogicalPlan): Option[String] = node match { + case Distinct(Project(list, child)) => + projectToSQL(list, child, isDistinct = true) + + case Project(list, child) => + projectToSQL(list, child, isDistinct = false) + + case Aggregate(groupingExprs, aggExprs, child) => + aggregateToSQL(groupingExprs, aggExprs, child) + + case Limit(limit, child) => + for { + childSQL <- toSQL(child) + limitSQL = limit.sql + } yield s"$childSQL LIMIT $limitSQL" + + case Filter(condition, child) => + for { + childSQL <- toSQL(child) + whereOrHaving = child match { + case _: Aggregate => "HAVING" + case _ => "WHERE" + } + conditionSQL = condition.sql + } yield s"$childSQL $whereOrHaving $conditionSQL" + + case Union(left, right) => + for { + leftSQL <- toSQL(left) + rightSQL <- toSQL(right) + } yield s"$leftSQL UNION ALL $rightSQL" + + // ParquetRelation converted from Hive metastore table + case Subquery(alias, LogicalRelation(r: ParquetRelation, _)) => + // There seems to be a bug related to `ParquetConversions` analysis rule. The problem is + // that, the metastore database name and table name are not always propagated to converted + // `ParquetRelation` instances via data source options. Here we use subquery alias as a + // workaround. + Some(s"`$alias`") + + case Subquery(alias, child) => + toSQL(child).map(childSQL => s"($childSQL) AS $alias") + + case Join(left, right, joinType, condition) => + for { + leftSQL <- toSQL(left) + rightSQL <- toSQL(right) + joinTypeSQL = joinType.sql + conditionSQL = condition.map(" ON " + _.sql).getOrElse("") + } yield s"$leftSQL $joinTypeSQL JOIN $rightSQL$conditionSQL" + + case MetastoreRelation(database, table, alias) => + val aliasSQL = alias.map(a => s" AS `$a`").getOrElse("") + Some(s"`$database`.`$table`$aliasSQL") + + case Sort(orders, _, RepartitionByExpression(partitionExprs, child, _)) + if orders.map(_.child) == partitionExprs => + for { + childSQL <- toSQL(child) + partitionExprsSQL = partitionExprs.map(_.sql).mkString(", ") + } yield s"$childSQL CLUSTER BY $partitionExprsSQL" + + case Sort(orders, global, child) => + for { + childSQL <- toSQL(child) + ordersSQL = orders.map { case SortOrder(e, dir) => s"${e.sql} ${dir.sql}" }.mkString(", ") + orderOrSort = if (global) "ORDER" else "SORT" + } yield s"$childSQL $orderOrSort BY $ordersSQL" + + case RepartitionByExpression(partitionExprs, child, _) => + for { + childSQL <- toSQL(child) + partitionExprsSQL = partitionExprs.map(_.sql).mkString(", ") + } yield s"$childSQL DISTRIBUTE BY $partitionExprsSQL" + + case OneRowRelation => + Some("") + + case _ => None + } + + object Canonicalizer extends RuleExecutor[LogicalPlan] { + override protected def batches: Seq[Batch] = Seq( + Batch("Canonicalizer", FixedPoint(100), + // The `WidenSetOperationTypes` analysis rule may introduce extra `Project`s over + // `Aggregate`s to perform type casting. This rule merges these `Project`s into + // `Aggregate`s. + ProjectCollapsing, + + // Used to handle other auxiliary `Project`s added by analyzer (e.g. + // `ResolveAggregateFunctions` rule) + RecoverScopingInfo + ) + ) + + object RecoverScopingInfo extends Rule[LogicalPlan] { + override def apply(tree: LogicalPlan): LogicalPlan = tree transform { + // This branch handles aggregate functions within HAVING clauses. For example: + // + // SELECT key FROM src GROUP BY key HAVING max(value) > "val_255" + // + // This kind of query results in query plans of the following form because of analysis rule + // `ResolveAggregateFunctions`: + // + // Project ... + // +- Filter ... + // +- Aggregate ... + // +- MetastoreRelation default, src, None + case plan @ Project(_, Filter(_, _: Aggregate)) => + wrapChildWithSubquery(plan) + + case plan @ Project(_, + _: Subquery | _: Filter | _: Join | _: MetastoreRelation | OneRowRelation | _: Limit + ) => plan + + case plan: Project => + wrapChildWithSubquery(plan) + } + + def wrapChildWithSubquery(project: Project): Project = project match { + case Project(projectList, child) => + val alias = SQLBuilder.newSubqueryName + val childAttributes = child.outputSet + val aliasedProjectList = projectList.map(_.transform { + case a: Attribute if childAttributes.contains(a) => + a.withQualifiers(alias :: Nil) + }.asInstanceOf[NamedExpression]) + + Project(aliasedProjectList, Subquery(alias, child)) + } + } + } +} + +object SQLBuilder { + private val nextSubqueryId = new AtomicLong(0) + + private def newSubqueryName: String = s"gen_subquery_${nextSubqueryId.getAndIncrement()}" +} 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 b1a6d0ab7df3c..e76c18fa528f3 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,30 +17,26 @@ package org.apache.spark.sql.hive -import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.util.Try import org.apache.hadoop.hive.ql.exec._ -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.GenericUDAFEvaluator.AggregationBuffer import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper -import org.apache.hadoop.hive.serde2.objectinspector.{ConstantObjectInspector, ObjectInspector, ObjectInspectorFactory} +import org.apache.hadoop.hive.ql.udf.generic._ +import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions +import org.apache.hadoop.hive.serde2.objectinspector.{ConstantObjectInspector, ObjectInspector, ObjectInspectorFactory} import org.apache.spark.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.catalyst.util.sequenceOption +import org.apache.spark.sql.catalyst.{InternalRow, analysis} import org.apache.spark.sql.hive.HiveShim._ import org.apache.spark.sql.hive.client.ClientWrapper import org.apache.spark.sql.types._ @@ -75,19 +71,19 @@ private[hive] class HiveFunctionRegistry( try { if (classOf[GenericUDFMacro].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUDF( - new HiveFunctionWrapper(functionClassName, functionInfo.getGenericUDF), children) + name, new HiveFunctionWrapper(functionClassName, functionInfo.getGenericUDF), children) } else if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveSimpleUDF(new HiveFunctionWrapper(functionClassName), children) + HiveSimpleUDF(name, new HiveFunctionWrapper(functionClassName), children) } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUDF(new HiveFunctionWrapper(functionClassName), children) + HiveGenericUDF(name, new HiveFunctionWrapper(functionClassName), children) } else if ( classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveUDAFFunction(new HiveFunctionWrapper(functionClassName), children) + HiveUDAFFunction(name, new HiveFunctionWrapper(functionClassName), children) } else if (classOf[UDAF].isAssignableFrom(functionInfo.getFunctionClass)) { HiveUDAFFunction( - new HiveFunctionWrapper(functionClassName), children, isUDAFBridgeRequired = true) + name, new HiveFunctionWrapper(functionClassName), children, isUDAFBridgeRequired = true) } else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) { - val udtf = HiveGenericUDTF(new HiveFunctionWrapper(functionClassName), children) + val udtf = HiveGenericUDTF(name, new HiveFunctionWrapper(functionClassName), children) udtf.elementTypes // Force it to check input data types. udtf } else { @@ -137,7 +133,8 @@ private[hive] class HiveFunctionRegistry( } } -private[hive] case class HiveSimpleUDF(funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) +private[hive] case class HiveSimpleUDF( + name: String, funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) extends Expression with HiveInspectors with CodegenFallback with Logging { override def deterministic: Boolean = isUDFDeterministic @@ -191,6 +188,8 @@ private[hive] case class HiveSimpleUDF(funcWrapper: HiveFunctionWrapper, childre override def toString: String = { s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" } + + override def sql: String = s"$name(${children.map(_.sql).mkString(", ")})" } // Adapter from Catalyst ExpressionResult to Hive DeferredObject @@ -205,7 +204,8 @@ private[hive] class DeferredObjectAdapter(oi: ObjectInspector, dataType: DataTyp override def get(): AnyRef = wrap(func(), oi, dataType) } -private[hive] case class HiveGenericUDF(funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) +private[hive] case class HiveGenericUDF( + name: String, funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) extends Expression with HiveInspectors with CodegenFallback with Logging { override def nullable: Boolean = true @@ -257,6 +257,8 @@ private[hive] case class HiveGenericUDF(funcWrapper: HiveFunctionWrapper, childr override def toString: String = { s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" } + + override def sql: String = s"$name(${children.map(_.sql).mkString(", ")})" } /** @@ -271,6 +273,7 @@ private[hive] case class HiveGenericUDF(funcWrapper: HiveFunctionWrapper, childr * user defined aggregations, which have clean semantics even in a partitioned execution. */ private[hive] case class HiveGenericUDTF( + name: String, funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) extends Generator with HiveInspectors with CodegenFallback { @@ -336,6 +339,8 @@ private[hive] case class HiveGenericUDTF( override def toString: String = { s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" } + + override def sql: String = s"$name(${children.map(_.sql).mkString(", ")})" } /** @@ -343,6 +348,7 @@ private[hive] case class HiveGenericUDTF( * performance a lot. */ private[hive] case class HiveUDAFFunction( + name: String, funcWrapper: HiveFunctionWrapper, children: Seq[Expression], isUDAFBridgeRequired: Boolean = false, @@ -427,5 +433,9 @@ private[hive] case class HiveUDAFFunction( override def supportsPartial: Boolean = false override val dataType: DataType = inspectorToDataType(returnInspector) -} + override def sql(isDistinct: Boolean): String = { + val distinct = if (isDistinct) "DISTINCT " else " " + s"$name($distinct${children.map(_.sql).mkString(", ")})" + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ExpressionSQLBuilderSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ExpressionSQLBuilderSuite.scala new file mode 100644 index 0000000000000..3a6eb57add4e3 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ExpressionSQLBuilderSuite.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.hive + +import java.sql.Timestamp + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{If, Literal} + +class ExpressionSQLBuilderSuite extends SQLBuilderTest { + test("literal") { + checkSQL(Literal("foo"), "\"foo\"") + checkSQL(Literal("\"foo\""), "\"\\\"foo\\\"\"") + checkSQL(Literal(1: Byte), "CAST(1 AS TINYINT)") + checkSQL(Literal(2: Short), "CAST(2 AS SMALLINT)") + checkSQL(Literal(4: Int), "4") + checkSQL(Literal(8: Long), "CAST(8 AS BIGINT)") + checkSQL(Literal(1.5F), "CAST(1.5 AS FLOAT)") + checkSQL(Literal(2.5D), "2.5") + checkSQL( + Literal(Timestamp.valueOf("2016-01-01 00:00:00")), + "TIMESTAMP('2016-01-01 00:00:00.0')") + // TODO tests for decimals + } + + test("binary comparisons") { + checkSQL('a.int === 'b.int, "(`a` = `b`)") + checkSQL('a.int <=> 'b.int, "(`a` <=> `b`)") + checkSQL('a.int !== 'b.int, "(NOT (`a` = `b`))") + + checkSQL('a.int < 'b.int, "(`a` < `b`)") + checkSQL('a.int <= 'b.int, "(`a` <= `b`)") + checkSQL('a.int > 'b.int, "(`a` > `b`)") + checkSQL('a.int >= 'b.int, "(`a` >= `b`)") + + checkSQL('a.int in ('b.int, 'c.int), "(`a` IN (`b`, `c`))") + checkSQL('a.int in (1, 2), "(`a` IN (1, 2))") + + checkSQL('a.int.isNull, "(`a` IS NULL)") + checkSQL('a.int.isNotNull, "(`a` IS NOT NULL)") + } + + test("logical operators") { + checkSQL('a.boolean && 'b.boolean, "(`a` AND `b`)") + checkSQL('a.boolean || 'b.boolean, "(`a` OR `b`)") + checkSQL(!'a.boolean, "(NOT `a`)") + checkSQL(If('a.boolean, 'b.int, 'c.int), "(IF(`a`, `b`, `c`))") + } + + test("arithmetic expressions") { + checkSQL('a.int + 'b.int, "(`a` + `b`)") + checkSQL('a.int - 'b.int, "(`a` - `b`)") + checkSQL('a.int * 'b.int, "(`a` * `b`)") + checkSQL('a.int / 'b.int, "(`a` / `b`)") + checkSQL('a.int % 'b.int, "(`a` % `b`)") + + checkSQL(-'a.int, "(-`a`)") + checkSQL(-('a.int + 'b.int), "(-(`a` + `b`))") + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala new file mode 100644 index 0000000000000..0e81acf532a03 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.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.sql.hive + +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.functions._ + +class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { + import testImplicits._ + + protected override def beforeAll(): Unit = { + sqlContext.range(10).write.saveAsTable("t0") + + sqlContext + .range(10) + .select('id as 'key, concat(lit("val_"), 'id) as 'value) + .write + .saveAsTable("t1") + + sqlContext.range(10).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd).write.saveAsTable("t2") + } + + override protected def afterAll(): Unit = { + sql("DROP TABLE IF EXISTS t0") + sql("DROP TABLE IF EXISTS t1") + sql("DROP TABLE IF EXISTS t2") + } + + private def checkHiveQl(hiveQl: String): Unit = { + val df = sql(hiveQl) + val convertedSQL = new SQLBuilder(df).toSQL + + if (convertedSQL.isEmpty) { + fail( + s"""Cannot convert the following HiveQL query plan back to SQL query string: + | + |# Original HiveQL query string: + |$hiveQl + | + |# Resolved query plan: + |${df.queryExecution.analyzed.treeString} + """.stripMargin) + } + + val sqlString = convertedSQL.get + try { + checkAnswer(sql(sqlString), df) + } catch { case cause: Throwable => + fail( + s"""Failed to execute converted SQL string or got wrong answer: + | + |# Converted SQL query string: + |$sqlString + | + |# Original HiveQL query string: + |$hiveQl + | + |# Resolved query plan: + |${df.queryExecution.analyzed.treeString} + """.stripMargin, + cause) + } + } + + test("in") { + checkHiveQl("SELECT id FROM t0 WHERE id IN (1, 2, 3)") + } + + test("aggregate function in having clause") { + checkHiveQl("SELECT COUNT(value) FROM t1 GROUP BY key HAVING MAX(key) > 0") + } + + test("aggregate function in order by clause") { + checkHiveQl("SELECT COUNT(value) FROM t1 GROUP BY key ORDER BY MAX(key)") + } + + // TODO Fix name collision introduced by ResolveAggregateFunction analysis rule + // When there are multiple aggregate functions in ORDER BY clause, all of them are extracted into + // Aggregate operator and aliased to the same name "aggOrder". This is OK for normal query + // execution since these aliases have different expression ID. But this introduces name collision + // when converting resolved plans back to SQL query strings as expression IDs are stripped. + ignore("aggregate function in order by clause with multiple order keys") { + checkHiveQl("SELECT COUNT(value) FROM t1 GROUP BY key ORDER BY key, MAX(key)") + } + + test("type widening in union") { + checkHiveQl("SELECT id FROM t0 UNION ALL SELECT CAST(id AS INT) AS id FROM t0") + } + + test("case") { + checkHiveQl("SELECT CASE WHEN id % 2 > 0 THEN 0 WHEN id % 2 = 0 THEN 1 END FROM t0") + } + + test("case with else") { + checkHiveQl("SELECT CASE WHEN id % 2 > 0 THEN 0 ELSE 1 END FROM t0") + } + + test("case with key") { + checkHiveQl("SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' END FROM t0") + } + + test("case with key and else") { + checkHiveQl("SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' ELSE 'baz' END FROM t0") + } + + test("select distinct without aggregate functions") { + checkHiveQl("SELECT DISTINCT id FROM t0") + } + + test("cluster by") { + checkHiveQl("SELECT id FROM t0 CLUSTER BY id") + } + + test("distribute by") { + checkHiveQl("SELECT id FROM t0 DISTRIBUTE BY id") + } + + test("distribute by with sort by") { + checkHiveQl("SELECT id FROM t0 DISTRIBUTE BY id SORT BY id") + } + + test("distinct aggregation") { + checkHiveQl("SELECT COUNT(DISTINCT id) FROM t0") + } + + // TODO Enable this + // Query plans transformed by DistinctAggregationRewriter are not recognized yet + ignore("distinct and non-distinct aggregation") { + checkHiveQl("SELECT a, COUNT(DISTINCT b), COUNT(DISTINCT c), SUM(d) FROM t2 GROUP BY a") + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.scala new file mode 100644 index 0000000000000..cf4a3fdd88806 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.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 + +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.{DataFrame, QueryTest} + +abstract class SQLBuilderTest extends QueryTest with TestHiveSingleton { + protected def checkSQL(e: Expression, expectedSQL: String): Unit = { + val actualSQL = e.sql + try { + assert(actualSQL === expectedSQL) + } catch { + case cause: Throwable => + fail( + s"""Wrong SQL generated for the following expression: + | + |${e.prettyName} + | + |$cause + """.stripMargin) + } + } + + protected def checkSQL(plan: LogicalPlan, expectedSQL: String): Unit = { + val maybeSQL = new SQLBuilder(plan, hiveContext).toSQL + + if (maybeSQL.isEmpty) { + fail( + s"""Cannot convert the following logical query plan to SQL: + | + |${plan.treeString} + """.stripMargin) + } + + val actualSQL = maybeSQL.get + + try { + assert(actualSQL === expectedSQL) + } catch { + case cause: Throwable => + fail( + s"""Wrong SQL generated for the following logical query plan: + | + |${plan.treeString} + | + |$cause + """.stripMargin) + } + + checkAnswer(sqlContext.sql(actualSQL), new DataFrame(sqlContext, plan)) + } + + protected def checkSQL(df: DataFrame, expectedSQL: String): Unit = { + checkSQL(df.queryExecution.analyzed, expectedSQL) + } +} 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 d7e8ebc8d312f..57358a07840e2 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 @@ -27,9 +27,10 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.{ExplainCommand, SetCommand} import org.apache.spark.sql.execution.datasources.DescribeCommand +import org.apache.spark.sql.execution.{ExplainCommand, SetCommand} import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.{InsertIntoHiveTable => LogicalInsertIntoHiveTable, SQLBuilder} /** * Allows the creations of tests that execute the same query against both hive @@ -130,6 +131,28 @@ abstract class HiveComparisonTest new java.math.BigInteger(1, digest.digest).toString(16) } + /** Used for testing [[SQLBuilder]] */ + private var numConvertibleQueries: Int = 0 + private var numTotalQueries: Int = 0 + + override protected def afterAll(): Unit = { + logInfo({ + val percentage = if (numTotalQueries > 0) { + numConvertibleQueries.toDouble / numTotalQueries * 100 + } else { + 0D + } + + s"""SQLBuiler statistics: + |- Total query number: $numTotalQueries + |- Number of convertible queries: $numConvertibleQueries + |- Percentage of convertible queries: $percentage% + """.stripMargin + }) + + super.afterAll() + } + protected def prepareAnswer( hiveQuery: TestHive.type#QueryExecution, answer: Seq[String]): Seq[String] = { @@ -372,8 +395,49 @@ abstract class HiveComparisonTest // Run w/ catalyst val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => - val query = new TestHive.QueryExecution(queryString) - try { (query, prepareAnswer(query, query.stringResult())) } catch { + var query: TestHive.QueryExecution = null + try { + query = { + val originalQuery = new TestHive.QueryExecution(queryString) + val containsCommands = originalQuery.analyzed.collectFirst { + case _: Command => () + case _: LogicalInsertIntoHiveTable => () + }.nonEmpty + + if (containsCommands) { + originalQuery + } else { + numTotalQueries += 1 + new SQLBuilder(originalQuery.analyzed, TestHive).toSQL.map { sql => + numConvertibleQueries += 1 + logInfo( + s""" + |### Running SQL generation round-trip test {{{ + |${originalQuery.analyzed.treeString} + |Original SQL: + |$queryString + | + |Generated SQL: + |$sql + |}}} + """.stripMargin.trim) + new TestHive.QueryExecution(sql) + }.getOrElse { + logInfo( + s""" + |### Cannot convert the following logical plan back to SQL {{{ + |${originalQuery.analyzed.treeString} + |Original SQL: + |$queryString + |}}} + """.stripMargin.trim) + originalQuery + } + } + } + + (query, prepareAnswer(query, query.stringResult())) + } catch { case e: Throwable => val errorMessage = s""" 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 fa99289b41971..4659d745fe78b 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 @@ -60,6 +60,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { TimeZone.setDefault(originalTimeZone) Locale.setDefault(originalLocale) sql("DROP TEMPORARY FUNCTION udtf_count2") + super.afterAll() } test("SPARK-4908: concurrent hive native commands") { From 1fdf9bbd67b884f23150b651f0fefdab6ccf008a Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 8 Jan 2016 20:50:08 -0800 Subject: [PATCH 045/110] [SPARK-12730][TESTS] De-duplicate some test code in BlockManagerSuite This patch deduplicates some test code in BlockManagerSuite. I'm splitting this change off from a larger PR in order to make things easier to review. Author: Josh Rosen Closes #10667 from JoshRosen/block-mgr-tests-cleanup. --- .../spark/storage/BlockManagerSuite.scala | 88 ++++++------------- 1 file changed, 25 insertions(+), 63 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 21db3b1c9ffbd..67210e5d4c50e 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -505,38 +505,27 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("in-memory LRU storage") { - 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) - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3").isDefined, "a3 was not in store") - assert(store.getSingle("a1") === None, "a1 was in store") - assert(store.getSingle("a2").isDefined, "a2 was not in store") - // At this point a2 was gotten last, so LRU will getSingle rid of a3 - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) - assert(store.getSingle("a1").isDefined, "a1 was not in store") - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3") === None, "a3 was in store") + testInMemoryLRUStorage(StorageLevel.MEMORY_ONLY) } test("in-memory LRU storage with serialization") { + testInMemoryLRUStorage(StorageLevel.MEMORY_ONLY_SER) + } + + private def testInMemoryLRUStorage(storageLevel: StorageLevel): Unit = { 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) + store.putSingle("a1", a1, storageLevel) + store.putSingle("a2", a2, storageLevel) + store.putSingle("a3", a3, storageLevel) assert(store.getSingle("a2").isDefined, "a2 was not in store") assert(store.getSingle("a3").isDefined, "a3 was not in store") assert(store.getSingle("a1") === None, "a1 was in store") assert(store.getSingle("a2").isDefined, "a2 was not in store") // At this point a2 was gotten last, so LRU will getSingle rid of a3 - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER) + store.putSingle("a1", a1, storageLevel) assert(store.getSingle("a1").isDefined, "a1 was not in store") assert(store.getSingle("a2").isDefined, "a2 was not in store") assert(store.getSingle("a3") === None, "a3 was in store") @@ -618,62 +607,35 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("disk and memory storage") { - 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) - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3").isDefined, "a3 was not in store") - assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") - assert(store.getSingle("a1").isDefined, "a1 was not in store") - assert(store.memoryStore.getValues("a1").isDefined, "a1 was not in memory store") + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK, _.getSingle) } test("disk and memory storage with getLocalBytes") { - 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) - assert(store.getLocalBytes("a2").isDefined, "a2 was not in store") - assert(store.getLocalBytes("a3").isDefined, "a3 was not in store") - assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") - assert(store.getLocalBytes("a1").isDefined, "a1 was not in store") - assert(store.memoryStore.getValues("a1").isDefined, "a1 was not in memory store") + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK, _.getLocalBytes) } test("disk and memory storage with serialization") { - 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) - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3").isDefined, "a3 was not in store") - assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") - assert(store.getSingle("a1").isDefined, "a1 was not in store") - assert(store.memoryStore.getValues("a1").isDefined, "a1 was not in memory store") + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK_SER, _.getSingle) } test("disk and memory storage with serialization and getLocalBytes") { + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK_SER, _.getLocalBytes) + } + + def testDiskAndMemoryStorage( + storageLevel: StorageLevel, + accessMethod: BlockManager => BlockId => Option[_]): Unit = { 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) - assert(store.getLocalBytes("a2").isDefined, "a2 was not in store") - assert(store.getLocalBytes("a3").isDefined, "a3 was not in store") - assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") - assert(store.getLocalBytes("a1").isDefined, "a1 was not in store") + store.putSingle("a1", a1, storageLevel) + store.putSingle("a2", a2, storageLevel) + store.putSingle("a3", a3, storageLevel) + assert(accessMethod(store)("a2").isDefined, "a2 was not in store") + assert(accessMethod(store)("a3").isDefined, "a3 was not in store") + assert(store.memoryStore.getValues("a1").isEmpty, "a1 was in memory store") + assert(accessMethod(store)("a1").isDefined, "a1 was not in store") assert(store.memoryStore.getValues("a1").isDefined, "a1 was not in memory store") } From 090d691323063c436601943506baac3ec5255dd9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 8 Jan 2016 20:58:53 -0800 Subject: [PATCH 046/110] [SPARK-4628][BUILD] Remove all non-Maven-Central repositories from build This patch removes all non-Maven-central repositories from Spark's build, thereby avoiding any risk of future build-breaks due to us accidentally depending on an artifact which is not present in an immutable public Maven repository. I tested this by running ``` build/mvn \ -Phive \ -Phive-thriftserver \ -Pkinesis-asl \ -Pspark-ganglia-lgpl \ -Pyarn \ dependency:go-offline ``` inside of a fresh Ubuntu Docker container with no Ivy or Maven caches (I did a similar test for SBT). Author: Josh Rosen Closes #10659 from JoshRosen/SPARK-4628. --- external/mqtt/pom.xml | 2 +- pom.xml | 87 ---------------------------------------- project/SparkBuild.scala | 7 +++- project/plugins.sbt | 6 --- 4 files changed, 7 insertions(+), 95 deletions(-) diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index b3ba72a0087ad..d3a2bf5825b08 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -51,7 +51,7 @@ org.eclipse.paho org.eclipse.paho.client.mqttv3 - 1.0.1 + 1.0.2 org.scalacheck diff --git a/pom.xml b/pom.xml index 9c975a45f8d23..0eac212754320 100644 --- a/pom.xml +++ b/pom.xml @@ -226,93 +226,6 @@ false - - apache-repo - Apache Repository - https://repository.apache.org/content/repositories/releases - - true - - - false - - - - jboss-repo - JBoss Repository - https://repository.jboss.org/nexus/content/repositories/releases - - true - - - false - - - - mqtt-repo - MQTT Repository - https://repo.eclipse.org/content/repositories/paho-releases - - true - - - false - - - - cloudera-repo - Cloudera Repository - https://repository.cloudera.com/artifactory/cloudera-repos - - true - - - false - - - - spark-hive-staging - Staging Repo for Hive 1.2.1 (Spark Version) - https://oss.sonatype.org/content/repositories/orgspark-project-1113 - - true - - - - mapr-repo - MapR Repository - http://repository.mapr.com/maven/ - - true - - - false - - - - - spring-releases - Spring Release Repository - https://repo.spring.io/libs-release - - false - - - false - - - - - twttr-repo - Twttr Repository - http://maven.twttr.com - - true - - - false - - diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5d4f19ab14a29..4c34c888cfd5e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -141,7 +141,12 @@ object SparkBuild extends PomBuild { publishMavenStyle := true, unidocGenjavadocVersion := "0.9-spark0", - resolvers += Resolver.mavenLocal, + // Override SBT's default resolvers: + resolvers := Seq( + DefaultMavenRepository, + Resolver.mavenLocal + ), + externalResolvers := resolvers.value, 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) diff --git a/project/plugins.sbt b/project/plugins.sbt index 15ba3a36d51ca..822a7c4a82d5e 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -1,9 +1,3 @@ -resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases"))(Resolver.ivyStylePatterns) - -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 95cd5d95ce8aec8b2462204c791ba927326305ba Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 8 Jan 2016 21:48:06 -0800 Subject: [PATCH 047/110] [SPARK-12577] [SQL] Better support of parentheses in partition by and order by clause of window function's over clause JIRA: https://issues.apache.org/jira/browse/SPARK-12577 Author: Liang-Chi Hsieh Closes #10620 from viirya/fix-parentheses. --- .../sql/catalyst/parser/ExpressionParser.g | 7 +++- .../spark/sql/catalyst/CatalystQlSuite.scala | 36 +++++++++++++------ 2 files changed, 32 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g index cad770122d150..aabb5d49582c8 100644 --- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g @@ -223,7 +223,12 @@ precedenceUnaryPrefixExpression ; precedenceUnarySuffixExpression - : precedenceUnaryPrefixExpression (a=KW_IS nullCondition)? + : + ( + (LPAREN precedenceUnaryPrefixExpression RPAREN) => LPAREN precedenceUnaryPrefixExpression (a=KW_IS nullCondition)? RPAREN + | + precedenceUnaryPrefixExpression (a=KW_IS nullCondition)? + ) -> {$a != null}? ^(TOK_FUNCTION nullCondition precedenceUnaryPrefixExpression) -> precedenceUnaryPrefixExpression ; diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala index 30978d9b49e2b..d7204c3488313 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala @@ -20,17 +20,33 @@ package org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.plans.PlanTest class CatalystQlSuite extends PlanTest { + val parser = new CatalystQl() test("parse union/except/intersect") { - val paresr = new CatalystQl() - paresr.createPlan("select * from t1 union all select * from t2") - paresr.createPlan("select * from t1 union distinct select * from t2") - paresr.createPlan("select * from t1 union select * from t2") - paresr.createPlan("select * from t1 except select * from t2") - paresr.createPlan("select * from t1 intersect select * from t2") - paresr.createPlan("(select * from t1) union all (select * from t2)") - paresr.createPlan("(select * from t1) union distinct (select * from t2)") - paresr.createPlan("(select * from t1) union (select * from t2)") - paresr.createPlan("select * from ((select * from t1) union (select * from t2)) t") + parser.createPlan("select * from t1 union all select * from t2") + parser.createPlan("select * from t1 union distinct select * from t2") + parser.createPlan("select * from t1 union select * from t2") + parser.createPlan("select * from t1 except select * from t2") + parser.createPlan("select * from t1 intersect select * from t2") + parser.createPlan("(select * from t1) union all (select * from t2)") + parser.createPlan("(select * from t1) union distinct (select * from t2)") + parser.createPlan("(select * from t1) union (select * from t2)") + parser.createPlan("select * from ((select * from t1) union (select * from t2)) t") + } + + test("window function: better support of parentheses") { + parser.createPlan("select sum(product + 1) over (partition by ((1) + (product / 2)) " + + "order by 2) from windowData") + parser.createPlan("select sum(product + 1) over (partition by (1 + (product / 2)) " + + "order by 2) from windowData") + parser.createPlan("select sum(product + 1) over (partition by ((product / 2) + 1) " + + "order by 2) from windowData") + + parser.createPlan("select sum(product + 1) over (partition by ((product) + (1)) order by 2) " + + "from windowData") + parser.createPlan("select sum(product + 1) over (partition by ((product) + 1) order by 2) " + + "from windowData") + parser.createPlan("select sum(product + 1) over (partition by (product + (1)) order by 2) " + + "from windowData") } } From 3d77cffec093bed4d330969f1a996f3358b9a772 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Sat, 9 Jan 2016 12:29:51 +0530 Subject: [PATCH 048/110] [SPARK-12645][SPARKR] SparkR support hash function Add ```hash``` function for SparkR ```DataFrame```. Author: Yanbo Liang Closes #10597 from yanboliang/spark-12645. --- R/pkg/NAMESPACE | 1 + R/pkg/R/functions.R | 20 ++++++++++++++++++++ R/pkg/R/generics.R | 4 ++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- 4 files changed, 26 insertions(+), 1 deletion(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index beacc39500aaa..34be7f0ebd752 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -130,6 +130,7 @@ exportMethods("%in%", "count", "countDistinct", "crc32", + "hash", "cume_dist", "date_add", "date_format", diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index df36bc869acb4..9bb7876b384ce 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -340,6 +340,26 @@ setMethod("crc32", column(jc) }) +#' hash +#' +#' Calculates the hash code of given columns, and returns the result as a int column. +#' +#' @rdname hash +#' @name hash +#' @family misc_funcs +#' @export +#' @examples \dontrun{hash(df$c)} +setMethod("hash", + signature(x = "Column"), + function(x, ...) { + jcols <- lapply(list(x, ...), function (x) { + stopifnot(class(x) == "Column") + x@jc + }) + jc <- callJStatic("org.apache.spark.sql.functions", "hash", jcols) + column(jc) + }) + #' dayofmonth #' #' Extracts the day of the month as an integer from a given date/timestamp/string. diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index ba6861709754d..5ba68e3a4f378 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -736,6 +736,10 @@ setGeneric("countDistinct", function(x, ...) { standardGeneric("countDistinct") #' @export setGeneric("crc32", function(x) { standardGeneric("crc32") }) +#' @rdname hash +#' @export +setGeneric("hash", function(x, ...) { standardGeneric("hash") }) + #' @rdname cume_dist #' @export setGeneric("cume_dist", function(x) { standardGeneric("cume_dist") }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index eaf60beda3473..97625b94a0e23 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -922,7 +922,7 @@ test_that("column functions", { c <- column("a") c1 <- abs(c) + acos(c) + approxCountDistinct(c) + ascii(c) + asin(c) + atan(c) c2 <- avg(c) + base64(c) + bin(c) + bitwiseNOT(c) + cbrt(c) + ceil(c) + cos(c) - c3 <- cosh(c) + count(c) + crc32(c) + exp(c) + c3 <- cosh(c) + count(c) + crc32(c) + hash(c) + exp(c) c4 <- explode(c) + expm1(c) + factorial(c) + first(c) + floor(c) + hex(c) c5 <- hour(c) + initcap(c) + last(c) + last_day(c) + length(c) c6 <- log(c) + (c) + log1p(c) + log2(c) + lower(c) + ltrim(c) + max(c) + md5(c) From b23c4521f5df905e4fe4d79dd5b670286e2697f7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 9 Jan 2016 11:21:58 -0800 Subject: [PATCH 049/110] [SPARK-12340] Fix overflow in various take functions. This is a follow-up for the original patch #10562. Author: Reynold Xin Closes #10670 from rxin/SPARK-12340. --- .../scala/org/apache/spark/rdd/AsyncRDDActions.scala | 8 ++++---- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 4 ++-- .../test/scala/org/apache/spark/rdd/RDDSuite.scala | 4 ++++ .../org/apache/spark/sql/execution/SparkPlan.scala | 7 +++---- .../scala/org/apache/spark/sql/DataFrameSuite.scala | 6 ++++++ .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 12 ------------ 6 files changed, 19 insertions(+), 22 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 94719a4572ef6..7de9df1e489fb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -77,7 +77,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi This implementation is non-blocking, asynchronously handling the results of each job and triggering the next job using callbacks on futures. */ - def continue(partsScanned: Long)(implicit jobSubmitter: JobSubmitter) : Future[Seq[T]] = + def continue(partsScanned: Int)(implicit jobSubmitter: JobSubmitter): Future[Seq[T]] = if (results.size >= num || partsScanned >= totalParts) { Future.successful(results.toSeq) } else { @@ -99,7 +99,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi } val left = num - results.size - val p = partsScanned.toInt until math.min(partsScanned + numPartsToTry, totalParts).toInt + val p = partsScanned.until(math.min(partsScanned + numPartsToTry, totalParts).toInt) val buf = new Array[Array[T]](p.size) self.context.setCallSite(callSite) @@ -109,13 +109,13 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi p, (index: Int, data: Array[T]) => buf(index) = data, Unit) - job.flatMap {_ => + job.flatMap { _ => buf.foreach(results ++= _.take(num - results.size)) continue(partsScanned + p.size) } } - new ComplexFutureAction[Seq[T]](continue(0L)(_)) + new ComplexFutureAction[Seq[T]](continue(0)(_)) } /** 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 e25657cc109be..de7102f5b6245 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1190,7 +1190,7 @@ abstract class RDD[T: ClassTag]( } else { val buf = new ArrayBuffer[T] val totalParts = this.partitions.length - var partsScanned = 0L + var partsScanned = 0 while (buf.size < num && partsScanned < totalParts) { // The number of partitions to try in this iteration. It is ok for this number to be // greater than totalParts because we actually cap it at totalParts in runJob. @@ -1209,7 +1209,7 @@ abstract class RDD[T: ClassTag]( } val left = num - buf.size - val p = partsScanned.toInt until math.min(partsScanned + numPartsToTry, totalParts).toInt + val p = partsScanned.until(math.min(partsScanned + numPartsToTry, totalParts).toInt) val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, p) res.foreach(buf ++= _.take(num - buf.size)) 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 24acbed4d7258..ef2ed445005d3 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -482,6 +482,10 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assert(nums.take(501) === (1 to 501).toArray) assert(nums.take(999) === (1 to 999).toArray) assert(nums.take(1000) === (1 to 999).toArray) + + nums = sc.parallelize(1 to 2, 2) + assert(nums.take(2147483638).size === 2) + assert(nums.takeAsync(2147483638).get.size === 2) } test("top with predefined ordering") { 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 21a6fba9078df..2355de3d05865 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 @@ -165,7 +165,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ val buf = new ArrayBuffer[InternalRow] val totalParts = childRDD.partitions.length - var partsScanned = 0L + var partsScanned = 0 while (buf.size < n && partsScanned < totalParts) { // The number of partitions to try in this iteration. It is ok for this number to be // greater than totalParts because we actually cap it at totalParts in runJob. @@ -183,10 +183,9 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ numPartsToTry = math.max(0, numPartsToTry) // guard against negative num of partitions val left = n - buf.size - val p = partsScanned.toInt until math.min(partsScanned + numPartsToTry, totalParts).toInt + val p = partsScanned.until(math.min(partsScanned + numPartsToTry, totalParts).toInt) val sc = sqlContext.sparkContext - val res = - sc.runJob(childRDD, (it: Iterator[InternalRow]) => it.take(left).toArray, p) + val res = sc.runJob(childRDD, (it: Iterator[InternalRow]) => it.take(left).toArray, p) res.foreach(buf ++= _.take(n - buf.size)) partsScanned += p.size diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index ade1391ecd74a..983dfbdedeefe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -308,6 +308,12 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { checkAnswer( mapData.toDF().limit(1), mapData.take(1).map(r => Row.fromSeq(r.productIterator.toSeq))) + + // SPARK-12340: overstep the bounds of Int in SparkPlan.executeTake + checkAnswer( + sqlContext.range(2).limit(2147483638), + Row(0) :: Row(1) :: Nil + ) } test("except") { 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 bd987ae1bb03a..5de0979606b88 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 @@ -2067,16 +2067,4 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { ) } } - - test("SPARK-12340: overstep the bounds of Int in SparkPlan.executeTake") { - val rdd = sqlContext.sparkContext.parallelize(1 to 3 , 3 ) - rdd.toDF("key").registerTempTable("spark12340") - checkAnswer( - sql("select key from spark12340 limit 2147483638"), - Row(1) :: Row(2) :: Row(3) :: Nil - ) - assert(rdd.take(2147483638).size === 3) - assert(rdd.takeAsync(2147483638).get.size === 3) - } - } From 3efd106e5cc1312bfba693a694ed33a3609a6741 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 9 Jan 2016 20:25:28 -0800 Subject: [PATCH 050/110] Close #10665 From 5b0d544339ef02fc25c816b6d6841031ef3902c2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 9 Jan 2016 20:28:20 -0800 Subject: [PATCH 051/110] [SPARK-12735] Consolidate & move spark-ec2 to AMPLab managed repository. Author: Reynold Xin Closes #10673 from rxin/SPARK-12735. --- .gitignore | 1 - dev/create-release/release-tag.sh | 3 - dev/create-release/releaseutils.py | 1 - dev/lint-python | 2 +- dev/sparktestsupport/modules.py | 9 - docs/_layouts/global.html | 2 - docs/cluster-overview.md | 2 - docs/ec2-scripts.md | 192 --- docs/index.md | 5 +- ec2/README | 4 - .../root/spark-ec2/ec2-variables.sh | 34 - ec2/spark-ec2 | 25 - ec2/spark_ec2.py | 1530 ----------------- make-distribution.sh | 1 - 14 files changed, 3 insertions(+), 1808 deletions(-) delete mode 100644 docs/ec2-scripts.md delete mode 100644 ec2/README delete mode 100644 ec2/deploy.generic/root/spark-ec2/ec2-variables.sh delete mode 100755 ec2/spark-ec2 delete mode 100755 ec2/spark_ec2.py diff --git a/.gitignore b/.gitignore index 07524bc429e92..8ecf536e79a5f 100644 --- a/.gitignore +++ b/.gitignore @@ -60,7 +60,6 @@ dev/create-release/*final spark-*-bin-*.tgz unit-tests.log /lib/ -ec2/lib/ rat-results.txt scalastyle.txt scalastyle-output.xml diff --git a/dev/create-release/release-tag.sh b/dev/create-release/release-tag.sh index b0a3374becc6a..d404939d1caee 100755 --- a/dev/create-release/release-tag.sh +++ b/dev/create-release/release-tag.sh @@ -64,9 +64,6 @@ git commit -a -m "Preparing Spark release $RELEASE_TAG" echo "Creating tag $RELEASE_TAG at the head of $GIT_BRANCH" git tag $RELEASE_TAG -# TODO: It would be nice to do some verifications here -# i.e. check whether ec2 scripts have the new version - # Create next version $MVN versions:set -DnewVersion=$NEXT_VERSION | grep -v "no value" # silence logs git commit -a -m "Preparing development version $NEXT_VERSION" diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py index 7f152b7f53559..5d0ac16b3b0a1 100755 --- a/dev/create-release/releaseutils.py +++ b/dev/create-release/releaseutils.py @@ -159,7 +159,6 @@ def get_commits(tag): "build": CORE_COMPONENT, "deploy": CORE_COMPONENT, "documentation": CORE_COMPONENT, - "ec2": "EC2", "examples": CORE_COMPONENT, "graphx": "GraphX", "input/output": CORE_COMPONENT, diff --git a/dev/lint-python b/dev/lint-python index 0b97213ae3dff..1765a07d2f22b 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -19,7 +19,7 @@ SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" SPARK_ROOT_DIR="$(dirname "$SCRIPT_DIR")" -PATHS_TO_CHECK="./python/pyspark/ ./ec2/spark_ec2.py ./examples/src/main/python/ ./dev/sparktestsupport" +PATHS_TO_CHECK="./python/pyspark/ ./examples/src/main/python/ ./dev/sparktestsupport" PATHS_TO_CHECK="$PATHS_TO_CHECK ./dev/run-tests.py ./python/run-tests.py ./dev/run-tests-jenkins.py" PEP8_REPORT_PATH="$SPARK_ROOT_DIR/dev/pep8-report.txt" PYLINT_REPORT_PATH="$SPARK_ROOT_DIR/dev/pylint-report.txt" diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 47cd600bd18a4..1fc6596164124 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -406,15 +406,6 @@ def contains_file(self, filename): should_run_build_tests=True ) -ec2 = Module( - name="ec2", - dependencies=[], - source_file_regexes=[ - "ec2/", - ] -) - - yarn = Module( name="yarn", dependencies=[], diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 62d75eff71057..d493f62f0e578 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -98,8 +98,6 @@
  • Spark Standalone
  • Mesos
  • YARN
  • -
  • -
  • Amazon EC2
  • diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index faaf154d243f5..2810112f5294e 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -53,8 +53,6 @@ The system currently supports three cluster managers: and service applications. * [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2. -In addition, Spark's [EC2 launch scripts](ec2-scripts.html) make it easy to launch a standalone -cluster on Amazon EC2. # Submitting Applications diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md deleted file mode 100644 index 7f60f82b966fe..0000000000000 --- a/docs/ec2-scripts.md +++ /dev/null @@ -1,192 +0,0 @@ ---- -layout: global -title: Running Spark on EC2 ---- - -The `spark-ec2` script, located in Spark's `ec2` directory, allows you -to launch, manage and shut down Spark clusters on Amazon EC2. It automatically -sets up Spark and HDFS on the cluster for you. This guide describes -how to use `spark-ec2` to launch clusters, how to run jobs on them, and how -to shut them down. It assumes you've already signed up for an EC2 account -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 -`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. - - -# Before You Start - -- Create an Amazon EC2 key pair for yourself. This can be done by - logging into your Amazon Web Services account through the [AWS - console](http://aws.amazon.com/console/), clicking Key Pairs on the - left sidebar, and creating and downloading a key. Make sure that you - set the permissions for the private key file to `600` (i.e. only you - can read and write it) so that `ssh` will work. -- Whenever you want to use the `spark-ec2` script, set the environment - variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` to your - Amazon EC2 access key ID and secret access key. These can be - obtained from the [AWS homepage](http://aws.amazon.com/) by clicking - Account \> Security Credentials \> Access Credentials. - -# Launching a Cluster - -- Go into the `ec2` directory in the release of Spark you downloaded. -- Run - `./spark-ec2 -k -i -s launch `, - where `` is the name of your EC2 key pair (that you gave it - when you created it), `` is the private key file for your - 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 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`). - -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 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 -instances. The default region is `us-east-1`. -- `--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=` 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=` 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=` 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. -- `--spark-git-repo=` will let you run a custom version of - Spark that is built from the given git repository. By default, the - [Apache Github mirror](https://github.com/apache/spark) will be used. - When using a custom Spark version, `--spark-version` must be set to git - commit hash, such as 317e114, instead of a version number. -- If one of your launches fails due to e.g. not having the right -permissions on your private key file, you can run `launch` with the -`--resume` option to restart the setup process on an existing cluster. - -# Launching a Cluster in a VPC - -- Run - `./spark-ec2 -k -i -s --vpc-id= --subnet-id= launch `, - where `` is the name of your EC2 key pair (that you gave it - when you created it), `` is the private key file for your - key pair, `` is the number of slave nodes to launch (try - 1 at first), `` is the name of your VPC, `` is the - name of your subnet, 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 --vpc-id=vpc-a28d24c7 --subnet-id=subnet-4eb27b39 --spark-version=1.1.0 launch my-spark-cluster - ``` - -# Running Applications - -- Go into the `ec2` directory in the release of Spark you downloaded. -- Run `./spark-ec2 -k -i login ` to - SSH into the cluster, where `` and `` are as - above. (This is just for convenience; you could also use - the EC2 console.) -- To deploy code or data within your cluster, you can log in and use the - provided script `~/spark-ec2/copy-dir`, which, - given a directory path, RSYNCs it to the same location on all the slaves. -- If your application needs to access large datasets, the fastest way to do - that is to load them from Amazon S3 or an Amazon EBS device into an - instance of the Hadoop Distributed File System (HDFS) on your nodes. - The `spark-ec2` script already sets up a HDFS instance for you. It's - installed in `/root/ephemeral-hdfs`, and can be accessed using the - `bin/hadoop` script in that directory. Note that the data in this - HDFS goes away when you stop and restart a machine. -- There is also a *persistent HDFS* instance in - `/root/persistent-hdfs` that will keep data across cluster restarts. - Typically each node has relatively little space of persistent data - (about 3 GB), but you can use the `--ebs-vol-size` option to - `spark-ec2` to attach a persistent EBS volume to each node for - storing the persistent HDFS. -- Finally, if you get errors while running your application, look at the slave's logs - for that application inside of the scheduler work directory (/root/spark/work). You can - also view the status of the cluster using the web UI: `http://:8080`. - -# Configuration - -You can edit `/root/spark/conf/spark-env.sh` on each machine to set Spark configuration options, such -as JVM options. This file needs to be copied to **every machine** to reflect the change. The easiest way to -do this is to use a script we provide called `copy-dir`. First edit your `spark-env.sh` file on the master, -then run `~/spark-ec2/copy-dir /root/spark/conf` to RSYNC it to all the workers. - -The [configuration guide](configuration.html) describes the available configuration options. - -# Terminating a Cluster - -***Note that there is no way to recover data on EC2 nodes after shutting -them down! Make sure you have copied everything important off the nodes -before stopping them.*** - -- Go into the `ec2` directory in the release of Spark you downloaded. -- Run `./spark-ec2 destroy `. - -# Pausing and Restarting Clusters - -The `spark-ec2` script also supports pausing a cluster. In this case, -the VMs are stopped but not terminated, so they -***lose all data on ephemeral disks*** but keep the data in their -root partitions and their `persistent-hdfs`. Stopped machines will not -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 --region= stop `. -- To restart it later, run -`./spark-ec2 -i --region= start `. -- To ultimately destroy the cluster and stop consuming EBS space, run -`./spark-ec2 --region= destroy ` as described in the previous -section. - -# Limitations - -- Support for "cluster compute" nodes is limited -- there's no way to specify a - locality group. However, you can launch slave nodes in your - `-slaves` group manually and then use `spark-ec2 launch - --resume` to start a cluster with them. - -If you have a patch or suggestion for one of these limitations, feel free to -[contribute](contributing-to-spark.html) it! - -# 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`. 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/docs/index.md b/docs/index.md index ae26f97c86c21..9dfc52a2bdc9b 100644 --- a/docs/index.md +++ b/docs/index.md @@ -64,7 +64,7 @@ To run Spark interactively in a R interpreter, use `bin/sparkR`: ./bin/sparkR --master local[2] Example applications are also provided in R. For example, - + ./bin/spark-submit examples/src/main/r/dataframe.R # Launching on a Cluster @@ -73,7 +73,6 @@ The Spark [cluster mode overview](cluster-overview.html) explains the key concep Spark can run both by itself, or over several existing cluster managers. It currently provides several options for deployment: -* [Amazon EC2](ec2-scripts.html): our EC2 scripts let you launch a cluster in about 5 minutes * [Standalone Deploy Mode](spark-standalone.html): simplest way to deploy Spark on a private cluster * [Apache Mesos](running-on-mesos.html) * [Hadoop YARN](running-on-yarn.html) @@ -103,7 +102,7 @@ options for deployment: * [Cluster Overview](cluster-overview.html): overview of concepts and components when running on a cluster * [Submitting Applications](submitting-applications.html): packaging and deploying applications * Deployment modes: - * [Amazon EC2](ec2-scripts.html): scripts that let you launch a cluster on EC2 in about 5 minutes + * [Amazon EC2](https://github.com/amplab/spark-ec2): scripts that let you launch a cluster on EC2 in about 5 minutes * [Standalone Deploy Mode](spark-standalone.html): launch a standalone cluster quickly without a third-party cluster manager * [Mesos](running-on-mesos.html): deploy a private cluster using [Apache Mesos](http://mesos.apache.org) diff --git a/ec2/README b/ec2/README deleted file mode 100644 index 72434f24bf98d..0000000000000 --- a/ec2/README +++ /dev/null @@ -1,4 +0,0 @@ -This folder contains a script, spark-ec2, for launching Spark clusters on -Amazon EC2. Usage instructions are available online at: - -http://spark.apache.org/docs/latest/ec2-scripts.html diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh deleted file mode 100644 index 4f3e8da809f7f..0000000000000 --- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh +++ /dev/null @@ -1,34 +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. -# - -# These variables are automatically filled in by the spark-ec2 script. -export MASTERS="{{master_list}}" -export SLAVES="{{slave_list}}" -export HDFS_DATA_DIRS="{{hdfs_data_dirs}}" -export MAPRED_LOCAL_DIRS="{{mapred_local_dirs}}" -export SPARK_LOCAL_DIRS="{{spark_local_dirs}}" -export MODULES="{{modules}}" -export SPARK_VERSION="{{spark_version}}" -export TACHYON_VERSION="{{tachyon_version}}" -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}}" diff --git a/ec2/spark-ec2 b/ec2/spark-ec2 deleted file mode 100755 index 26e7d22655694..0000000000000 --- a/ec2/spark-ec2 +++ /dev/null @@ -1,25 +0,0 @@ -#!/bin/sh - -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Preserve the user's CWD so that relative paths are passed correctly to -#+ the underlying Python script. -SPARK_EC2_DIR="$(dirname "$0")" - -python -Wdefault "${SPARK_EC2_DIR}/spark_ec2.py" "$@" diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py deleted file mode 100755 index 19d5980560fef..0000000000000 --- a/ec2/spark_ec2.py +++ /dev/null @@ -1,1530 +0,0 @@ -#!/usr/bin/env python -# -*- coding: utf-8 -*- - -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT 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 __future__ import division, print_function, with_statement - -import codecs -import hashlib -import itertools -import logging -import os -import os.path -import pipes -import random -import shutil -import string -from stat import S_IRUSR -import subprocess -import sys -import tarfile -import tempfile -import textwrap -import time -import warnings -from datetime import datetime -from optparse import OptionParser -from sys import stderr - -if sys.version < "3": - from urllib2 import urlopen, Request, HTTPError -else: - from urllib.request import urlopen, Request - from urllib.error import HTTPError - raw_input = input - xrange = range - -SPARK_EC2_VERSION = "1.6.0" -SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) - -VALID_SPARK_VERSIONS = set([ - "0.7.3", - "0.8.0", - "0.8.1", - "0.9.0", - "0.9.1", - "0.9.2", - "1.0.0", - "1.0.1", - "1.0.2", - "1.1.0", - "1.1.1", - "1.2.0", - "1.2.1", - "1.3.0", - "1.3.1", - "1.4.0", - "1.4.1", - "1.5.0", - "1.5.1", - "1.5.2", - "1.6.0", -]) - -SPARK_TACHYON_MAP = { - "1.0.0": "0.4.1", - "1.0.1": "0.4.1", - "1.0.2": "0.4.1", - "1.1.0": "0.5.0", - "1.1.1": "0.5.0", - "1.2.0": "0.5.0", - "1.2.1": "0.5.0", - "1.3.0": "0.5.0", - "1.3.1": "0.5.0", - "1.4.0": "0.6.4", - "1.4.1": "0.6.4", - "1.5.0": "0.7.1", - "1.5.1": "0.7.1", - "1.5.2": "0.7.1", - "1.6.0": "0.8.2", -} - -DEFAULT_SPARK_VERSION = SPARK_EC2_VERSION -DEFAULT_SPARK_GITHUB_REPO = "https://github.com/apache/spark" - -# Default location to get the spark-ec2 scripts (and ami-list) from -DEFAULT_SPARK_EC2_GITHUB_REPO = "https://github.com/amplab/spark-ec2" -DEFAULT_SPARK_EC2_BRANCH = "branch-1.5" - - -def setup_external_libs(libs): - """ - Download external libraries from PyPI to SPARK_EC2_DIR/lib/ and prepend them to our PATH. - """ - PYPI_URL_PREFIX = "https://pypi.python.org/packages/source" - SPARK_EC2_LIB_DIR = os.path.join(SPARK_EC2_DIR, "lib") - - if not os.path.exists(SPARK_EC2_LIB_DIR): - print("Downloading external libraries that spark-ec2 needs from PyPI to {path}...".format( - path=SPARK_EC2_LIB_DIR - )) - print("This should be a one-time operation.") - os.mkdir(SPARK_EC2_LIB_DIR) - - for lib in libs: - versioned_lib_name = "{n}-{v}".format(n=lib["name"], v=lib["version"]) - lib_dir = os.path.join(SPARK_EC2_LIB_DIR, versioned_lib_name) - - if not os.path.isdir(lib_dir): - tgz_file_path = os.path.join(SPARK_EC2_LIB_DIR, versioned_lib_name + ".tar.gz") - print(" - Downloading {lib}...".format(lib=lib["name"])) - download_stream = urlopen( - "{prefix}/{first_letter}/{lib_name}/{lib_name}-{lib_version}.tar.gz".format( - prefix=PYPI_URL_PREFIX, - first_letter=lib["name"][:1], - lib_name=lib["name"], - lib_version=lib["version"] - ) - ) - with open(tgz_file_path, "wb") as tgz_file: - tgz_file.write(download_stream.read()) - with open(tgz_file_path, "rb") as tar: - if hashlib.md5(tar.read()).hexdigest() != lib["md5"]: - print("ERROR: Got wrong md5sum for {lib}.".format(lib=lib["name"]), file=stderr) - sys.exit(1) - tar = tarfile.open(tgz_file_path) - tar.extractall(path=SPARK_EC2_LIB_DIR) - tar.close() - os.remove(tgz_file_path) - print(" - Finished downloading {lib}.".format(lib=lib["name"])) - sys.path.insert(1, lib_dir) - - -# Only PyPI libraries are supported. -external_libs = [ - { - "name": "boto", - "version": "2.34.0", - "md5": "5556223d2d0cc4d06dd4829e671dcecd" - } -] - -setup_external_libs(external_libs) - -import boto -from boto.ec2.blockdevicemapping import BlockDeviceMapping, BlockDeviceType, EBSBlockDeviceType -from boto import ec2 - - -class UsageError(Exception): - pass - - -# Configure and parse our command-line arguments -def parse_args(): - parser = OptionParser( - prog="spark-ec2", - version="%prog {v}".format(v=SPARK_EC2_VERSION), - usage="%prog [options] \n\n" - + " can be: launch, destroy, login, stop, start, get-master, reboot-slaves") - - parser.add_option( - "-s", "--slaves", type="int", default=1, - help="Number of slaves to launch (default: %default)") - parser.add_option( - "-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") - parser.add_option( - "-i", "--identity-file", - help="SSH private key file to use for logging into instances") - parser.add_option( - "-p", "--profile", default=None, - help="If you have multiple profiles (AWS or boto config), you can configure " + - "additional, named profiles by using this option (default: %default)") - parser.add_option( - "-t", "--instance-type", 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="", - help="Master instance type (leave empty for same as instance-type)") - parser.add_option( - "-r", "--region", default="us-east-1", - help="EC2 region used to launch instances in, or to find them in (default: %default)") - parser.add_option( - "-z", "--zone", default="", - help="Availability zone to launch instances in, or 'all' to spread " + - "slaves across multiple (an additional $0.01/Gb for bandwidth" + - "between zones applies) (default: a single zone chosen at random)") - parser.add_option( - "-a", "--ami", - help="Amazon Machine Image ID to use") - parser.add_option( - "-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=DEFAULT_SPARK_GITHUB_REPO, - help="Github repo from which to checkout supplied commit hash (default: %default)") - parser.add_option( - "--spark-ec2-git-repo", - default=DEFAULT_SPARK_EC2_GITHUB_REPO, - help="Github repo from which to checkout spark-ec2 (default: %default)") - parser.add_option( - "--spark-ec2-git-branch", - default=DEFAULT_SPARK_EC2_BRANCH, - help="Github repo branch of spark-ec2 to use (default: %default)") - parser.add_option( - "--deploy-root-dir", - default=None, - help="A directory to copy into / on the first master. " + - "Must be absolute. Note that a trailing slash is handled as per rsync: " + - "If you omit it, the last directory of the --deploy-root-dir path will be created " + - "in / before copying its contents. If you append the trailing slash, " + - "the directory is not created and its contents are copied directly into /. " + - "(default: %default).") - parser.add_option( - "--hadoop-major-version", default="1", - help="Major version of Hadoop. Valid options are 1 (Hadoop 1.0.4), 2 (CDH 4.2.0), yarn " + - "(Hadoop 2.4.0) (default: %default)") - parser.add_option( - "-D", metavar="[ADDRESS:]PORT", dest="proxy_port", - help="Use SSH dynamic port forwarding to create a SOCKS proxy at " + - "the given local address (for use with login)") - parser.add_option( - "--resume", action="store_true", default=False, - help="Resume installation on a previously launched cluster " + - "(for debugging)") - parser.add_option( - "--ebs-vol-size", metavar="SIZE", type="int", default=0, - 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( - "--placement-group", type="string", default=None, - help="Which placement group to try and launch " + - "instances into. Assumes placement group is already " + - "created.") - parser.add_option( - "--swap", metavar="SWAP", type="int", 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: %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: %default)") - parser.add_option( - "--delete-groups", action="store_true", default=False, - 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") - parser.add_option( - "--worker-instances", type="int", default=1, - help="Number of instances per worker: variable SPARK_WORKER_INSTANCES. Not used if YARN " + - "is used as Hadoop major version (default: %default)") - parser.add_option( - "--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 AMIs interpret this as an initialization script)") - parser.add_option( - "--authorized-address", type="string", 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") - parser.add_option( - "--additional-tags", type="string", default="", - help="Additional tags to set on the machines; tags are comma-separated, while name and " + - "value are colon separated; ex: \"Task:MySparkProject,Env:production\"") - parser.add_option( - "--copy-aws-credentials", action="store_true", default=False, - help="Add AWS credentials to hadoop configuration to allow Spark to access S3") - parser.add_option( - "--subnet-id", default=None, - help="VPC subnet to launch instances in") - parser.add_option( - "--vpc-id", default=None, - help="VPC to launch instances in") - parser.add_option( - "--private-ips", action="store_true", default=False, - help="Use private IPs for instances rather than public if VPC/subnet " + - "requires that.") - parser.add_option( - "--instance-initiated-shutdown-behavior", default="stop", - choices=["stop", "terminate"], - help="Whether instances should terminate when shut down or just stop") - parser.add_option( - "--instance-profile-name", default=None, - help="IAM profile name to launch instances under") - - (opts, args) = parser.parse_args() - if len(args) != 2: - parser.print_help() - sys.exit(1) - (action, cluster_name) = args - - # Boto config check - # http://boto.cloudhackers.com/en/latest/boto_config_tut.html - home_dir = os.getenv('HOME') - if home_dir is None or not os.path.isfile(home_dir + '/.boto'): - if not os.path.isfile('/etc/boto.cfg'): - # If there is no boto config, check aws credentials - if not os.path.isfile(home_dir + '/.aws/credentials'): - if os.getenv('AWS_ACCESS_KEY_ID') is None: - print("ERROR: The environment variable AWS_ACCESS_KEY_ID must be set", - file=stderr) - sys.exit(1) - if os.getenv('AWS_SECRET_ACCESS_KEY') is None: - print("ERROR: The environment variable AWS_SECRET_ACCESS_KEY must be set", - file=stderr) - sys.exit(1) - return (opts, action, cluster_name) - - -# Get the EC2 security group of the given name, creating it if it doesn't exist -def get_or_make_group(conn, name, vpc_id): - groups = conn.get_all_security_groups() - group = [g for g in groups if g.name == name] - if len(group) > 0: - return group[0] - else: - print("Creating security group " + name) - return conn.create_security_group(name, "Spark EC2 group", vpc_id) - - -def get_validate_spark_version(version, repo): - if "." in version: - version = version.replace("v", "") - if version not in VALID_SPARK_VERSIONS: - print("Don't know about Spark version: {v}".format(v=version), file=stderr) - sys.exit(1) - return version - else: - github_commit_url = "{repo}/commit/{commit_hash}".format(repo=repo, commit_hash=version) - request = Request(github_commit_url) - request.get_method = lambda: 'HEAD' - try: - response = urlopen(request) - except HTTPError as e: - print("Couldn't validate Spark commit: {url}".format(url=github_commit_url), - file=stderr) - print("Received HTTP response code of {code}.".format(code=e.code), file=stderr) - sys.exit(1) - return version - - -# Source: http://aws.amazon.com/amazon-linux-ami/instance-type-matrix/ -# Last Updated: 2015-06-19 -# For easy maintainability, please keep this manually-inputted dictionary sorted by key. -EC2_INSTANCE_TYPES = { - "c1.medium": "pvm", - "c1.xlarge": "pvm", - "c3.large": "pvm", - "c3.xlarge": "pvm", - "c3.2xlarge": "pvm", - "c3.4xlarge": "pvm", - "c3.8xlarge": "pvm", - "c4.large": "hvm", - "c4.xlarge": "hvm", - "c4.2xlarge": "hvm", - "c4.4xlarge": "hvm", - "c4.8xlarge": "hvm", - "cc1.4xlarge": "hvm", - "cc2.8xlarge": "hvm", - "cg1.4xlarge": "hvm", - "cr1.8xlarge": "hvm", - "d2.xlarge": "hvm", - "d2.2xlarge": "hvm", - "d2.4xlarge": "hvm", - "d2.8xlarge": "hvm", - "g2.2xlarge": "hvm", - "g2.8xlarge": "hvm", - "hi1.4xlarge": "pvm", - "hs1.8xlarge": "pvm", - "i2.xlarge": "hvm", - "i2.2xlarge": "hvm", - "i2.4xlarge": "hvm", - "i2.8xlarge": "hvm", - "m1.small": "pvm", - "m1.medium": "pvm", - "m1.large": "pvm", - "m1.xlarge": "pvm", - "m2.xlarge": "pvm", - "m2.2xlarge": "pvm", - "m2.4xlarge": "pvm", - "m3.medium": "hvm", - "m3.large": "hvm", - "m3.xlarge": "hvm", - "m3.2xlarge": "hvm", - "m4.large": "hvm", - "m4.xlarge": "hvm", - "m4.2xlarge": "hvm", - "m4.4xlarge": "hvm", - "m4.10xlarge": "hvm", - "r3.large": "hvm", - "r3.xlarge": "hvm", - "r3.2xlarge": "hvm", - "r3.4xlarge": "hvm", - "r3.8xlarge": "hvm", - "t1.micro": "pvm", - "t2.micro": "hvm", - "t2.small": "hvm", - "t2.medium": "hvm", - "t2.large": "hvm", -} - - -def get_tachyon_version(spark_version): - return SPARK_TACHYON_MAP.get(spark_version, "") - - -# Attempt to resolve an appropriate AMI given the architecture and region of the request. -def get_spark_ami(opts): - if opts.instance_type in EC2_INSTANCE_TYPES: - instance_type = EC2_INSTANCE_TYPES[opts.instance_type] - else: - instance_type = "pvm" - print("Don't recognize %s, assuming type is pvm" % opts.instance_type, file=stderr) - - # URL prefix from which to fetch AMI information - ami_prefix = "{r}/{b}/ami-list".format( - r=opts.spark_ec2_git_repo.replace("https://github.com", "https://raw.github.com", 1), - b=opts.spark_ec2_git_branch) - - ami_path = "%s/%s/%s" % (ami_prefix, opts.region, instance_type) - reader = codecs.getreader("ascii") - try: - ami = reader(urlopen(ami_path)).read().strip() - except: - print("Could not resolve AMI at: " + ami_path, file=stderr) - sys.exit(1) - - print("Spark AMI: " + ami) - return ami - - -# Launch a cluster of the given name, by setting up its security groups, -# and then starting new instances in them. -# Returns a tuple of EC2 reservation objects for the master and slaves -# Fails if there already instances running in the cluster's groups. -def launch_cluster(conn, opts, cluster_name): - if opts.identity_file is None: - print("ERROR: Must provide an identity file (-i) for ssh connections.", file=stderr) - sys.exit(1) - - if opts.key_pair is None: - print("ERROR: Must provide a key pair name (-k) to use on instances.", file=stderr) - 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", opts.vpc_id) - slave_group = get_or_make_group(conn, cluster_name + "-slaves", opts.vpc_id) - authorized_address = opts.authorized_address - if master_group.rules == []: # Group was just now created - if opts.vpc_id is None: - master_group.authorize(src_group=master_group) - master_group.authorize(src_group=slave_group) - else: - master_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, - src_group=master_group) - master_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, - src_group=master_group) - master_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, - src_group=master_group) - master_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, - src_group=slave_group) - master_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, - src_group=slave_group) - master_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, - src_group=slave_group) - 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) - # Rstudio (GUI for R) needs port 8787 for web access - master_group.authorize('tcp', 8787, 8787, authorized_address) - # HDFS NFS gateway requires 111,2049,4242 for tcp & udp - master_group.authorize('tcp', 111, 111, authorized_address) - master_group.authorize('udp', 111, 111, authorized_address) - master_group.authorize('tcp', 2049, 2049, authorized_address) - master_group.authorize('udp', 2049, 2049, authorized_address) - master_group.authorize('tcp', 4242, 4242, authorized_address) - master_group.authorize('udp', 4242, 4242, authorized_address) - # RM in YARN mode uses 8088 - master_group.authorize('tcp', 8088, 8088, authorized_address) - if opts.ganglia: - master_group.authorize('tcp', 5080, 5080, authorized_address) - if slave_group.rules == []: # Group was just now created - if opts.vpc_id is None: - slave_group.authorize(src_group=master_group) - slave_group.authorize(src_group=slave_group) - else: - slave_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, - src_group=master_group) - slave_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, - src_group=master_group) - slave_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, - src_group=master_group) - slave_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, - src_group=slave_group) - slave_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, - src_group=slave_group) - slave_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, - src_group=slave_group) - 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 in our groups - 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("ERROR: There are already instances running in group %s or %s" % - (master_group.name, slave_group.name), file=stderr) - sys.exit(1) - - # Figure out Spark AMI - if opts.ami is None: - opts.ami = get_spark_ami(opts) - - # we use group ids to work around https://github.com/boto/boto/issues/350 - additional_group_ids = [] - if opts.additional_security_group: - additional_group_ids = [sg.id - for sg in conn.get_all_security_groups() - if opts.additional_security_group in (sg.name, sg.id)] - print("Launching instances...") - - try: - image = conn.get_all_images(image_ids=[opts.ami])[0] - except: - print("Could not find AMI " + opts.ami, file=stderr) - sys.exit(1) - - # 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: - 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.'): - for i in range(get_num_disks(opts.instance_type)): - dev = BlockDeviceType() - dev.ephemeral_name = 'ephemeral%d' % i - # The first ephemeral drive is /dev/sdb. - name = '/dev/sd' + string.ascii_letters[i + 1] - block_map[name] = dev - - # Launch slaves - if opts.spot_price is not None: - # Launch spot instances with the requested price - print("Requesting %d slaves as spot instances with price $%.3f" % - (opts.slaves, opts.spot_price)) - zones = get_zones(conn, opts) - num_zones = len(zones) - i = 0 - my_req_ids = [] - for zone in zones: - num_slaves_this_zone = get_partition(opts.slaves, num_zones, i) - slave_reqs = conn.request_spot_instances( - price=opts.spot_price, - image_id=opts.ami, - launch_group="launch-group-%s" % cluster_name, - placement=zone, - count=num_slaves_this_zone, - key_name=opts.key_pair, - security_group_ids=[slave_group.id] + additional_group_ids, - instance_type=opts.instance_type, - block_device_map=block_map, - subnet_id=opts.subnet_id, - placement_group=opts.placement_group, - user_data=user_data_content, - instance_profile_name=opts.instance_profile_name) - my_req_ids += [req.id for req in slave_reqs] - i += 1 - - print("Waiting for spot instances to be granted...") - try: - while True: - time.sleep(10) - reqs = conn.get_all_spot_instance_requests() - id_to_req = {} - for r in reqs: - id_to_req[r.id] = r - active_instance_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 len(active_instance_ids) == opts.slaves: - print("All %d slaves granted" % opts.slaves) - reservations = conn.get_all_reservations(active_instance_ids) - slave_nodes = [] - for r in reservations: - slave_nodes += r.instances - break - else: - print("%d of %d slaves granted, waiting longer" % ( - len(active_instance_ids), opts.slaves)) - except: - print("Canceling spot instance requests") - conn.cancel_spot_instance_requests(my_req_ids) - # Log a warning if any of these requests actually launched instances: - (master_nodes, slave_nodes) = get_existing_cluster( - conn, opts, cluster_name, die_on_error=False) - running = len(master_nodes) + len(slave_nodes) - if running: - print(("WARNING: %d instances are still running" % running), file=stderr) - sys.exit(0) - else: - # Launch non-spot instances - zones = get_zones(conn, opts) - num_zones = len(zones) - i = 0 - slave_nodes = [] - for zone in zones: - 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_group_ids=[slave_group.id] + additional_group_ids, - instance_type=opts.instance_type, - placement=zone, - min_count=num_slaves_this_zone, - max_count=num_slaves_this_zone, - block_device_map=block_map, - subnet_id=opts.subnet_id, - placement_group=opts.placement_group, - user_data=user_data_content, - instance_initiated_shutdown_behavior=opts.instance_initiated_shutdown_behavior, - instance_profile_name=opts.instance_profile_name) - slave_nodes += slave_res.instances - print("Launched {s} slave{plural_s} in {z}, regid = {r}".format( - s=num_slaves_this_zone, - plural_s=('' if num_slaves_this_zone == 1 else 's'), - z=zone, - r=slave_res.id)) - i += 1 - - # Launch or resume masters - if existing_masters: - print("Starting master...") - for inst in existing_masters: - if inst.state not in ["shutting-down", "terminated"]: - inst.start() - master_nodes = existing_masters - else: - master_type = opts.master_instance_type - if master_type == "": - master_type = opts.instance_type - if opts.zone == 'all': - opts.zone = random.choice(conn.get_all_zones()).name - master_res = image.run( - key_name=opts.key_pair, - security_group_ids=[master_group.id] + additional_group_ids, - instance_type=master_type, - placement=opts.zone, - min_count=1, - max_count=1, - block_device_map=block_map, - subnet_id=opts.subnet_id, - placement_group=opts.placement_group, - user_data=user_data_content, - instance_initiated_shutdown_behavior=opts.instance_initiated_shutdown_behavior, - instance_profile_name=opts.instance_profile_name) - - master_nodes = master_res.instances - print("Launched master in %s, regid = %s" % (zone, master_res.id)) - - # This wait time corresponds to SPARK-4983 - print("Waiting for AWS to propagate instance metadata...") - time.sleep(15) - - # Give the instances descriptive names and set additional tags - additional_tags = {} - if opts.additional_tags.strip(): - additional_tags = dict( - map(str.strip, tag.split(':', 1)) for tag in opts.additional_tags.split(',') - ) - - for master in master_nodes: - master.add_tags( - dict(additional_tags, Name='{cn}-master-{iid}'.format(cn=cluster_name, iid=master.id)) - ) - - for slave in slave_nodes: - slave.add_tags( - dict(additional_tags, Name='{cn}-slave-{iid}'.format(cn=cluster_name, iid=slave.id)) - ) - - # Return all the instances - return (master_nodes, slave_nodes) - - -def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): - """ - Get the EC2 instances in an existing cluster if available. - Returns a tuple of lists of EC2 instance objects for the masters and slaves. - """ - print("Searching for existing cluster {c} in region {r}...".format( - c=cluster_name, r=opts.region)) - - def get_instances(group_names): - """ - Get all non-terminated instances that belong to any of the provided security groups. - - EC2 reservation filters and instance states are documented here: - http://docs.aws.amazon.com/cli/latest/reference/ec2/describe-instances.html#options - """ - reservations = conn.get_all_reservations( - filters={"instance.group-name": group_names}) - instances = itertools.chain.from_iterable(r.instances for r in reservations) - return [i for i in instances if i.state not in ["shutting-down", "terminated"]] - - master_instances = get_instances([cluster_name + "-master"]) - slave_instances = get_instances([cluster_name + "-slaves"]) - - if any((master_instances, slave_instances)): - print("Found {m} master{plural_m}, {s} slave{plural_s}.".format( - m=len(master_instances), - plural_m=('' if len(master_instances) == 1 else 's'), - s=len(slave_instances), - plural_s=('' if len(slave_instances) == 1 else 's'))) - - if not master_instances and die_on_error: - print("ERROR: Could not find a master for cluster {c} in region {r}.".format( - c=cluster_name, r=opts.region), file=sys.stderr) - sys.exit(1) - - return (master_instances, slave_instances) - - -# 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 = get_dns_name(master_nodes[0], opts.private_ips) - if deploy_ssh_key: - print("Generating cluster's SSH key on master...") - key_setup = """ - [ -f ~/.ssh/id_rsa ] || - (ssh-keygen -q -t rsa -N '' -f ~/.ssh/id_rsa && - cat ~/.ssh/id_rsa.pub >> ~/.ssh/authorized_keys) - """ - ssh(master, opts, key_setup) - dot_ssh_tar = ssh_read(master, opts, ['tar', 'c', '.ssh']) - print("Transferring cluster's SSH key to slaves...") - for slave in slave_nodes: - slave_address = get_dns_name(slave, opts.private_ips) - print(slave_address) - ssh_write(slave_address, opts, ['tar', 'x'], dot_ssh_tar) - - modules = ['spark', 'ephemeral-hdfs', 'persistent-hdfs', - 'mapreduce', 'spark-standalone', 'tachyon', 'rstudio'] - - if opts.hadoop_major_version == "1": - modules = list(filter(lambda x: x != "mapreduce", modules)) - - if opts.ganglia: - modules.append('ganglia') - - # Clear SPARK_WORKER_INSTANCES if running on YARN - if opts.hadoop_major_version == "yarn": - opts.worker_instances = "" - - # NOTE: We should clone the repository before running deploy_files to - # prevent ec2-variables.sh from being overwritten - print("Cloning spark-ec2 scripts from {r}/tree/{b} on master...".format( - r=opts.spark_ec2_git_repo, b=opts.spark_ec2_git_branch)) - ssh( - host=master, - opts=opts, - command="rm -rf spark-ec2" - + " && " - + "git clone {r} -b {b} spark-ec2".format(r=opts.spark_ec2_git_repo, - b=opts.spark_ec2_git_branch) - ) - - print("Deploying files to master...") - deploy_files( - conn=conn, - root_dir=SPARK_EC2_DIR + "/" + "deploy.generic", - opts=opts, - master_nodes=master_nodes, - slave_nodes=slave_nodes, - modules=modules - ) - - if opts.deploy_root_dir is not None: - print("Deploying {s} to master...".format(s=opts.deploy_root_dir)) - deploy_user_files( - root_dir=opts.deploy_root_dir, - opts=opts, - master_nodes=master_nodes - ) - - print("Running setup on master...") - setup_spark_cluster(master, opts) - print("Done!") - - -def setup_spark_cluster(master, opts): - ssh(master, opts, "chmod u+x spark-ec2/setup.sh") - ssh(master, opts, "spark-ec2/setup.sh") - print("Spark standalone cluster started at http://%s:8080" % master) - - if opts.ganglia: - print("Ganglia started at http://%s:5080/ganglia" % master) - - -def is_ssh_available(host, opts, print_ssh_output=True): - """ - Check if SSH is available on a host. - """ - s = subprocess.Popen( - ssh_command(opts) + ['-t', '-t', '-o', 'ConnectTimeout=3', - '%s@%s' % (opts.user, host), stringify_command('true')], - stdout=subprocess.PIPE, - stderr=subprocess.STDOUT # we pipe stderr through stdout to preserve output order - ) - cmd_output = s.communicate()[0] # [1] is stderr, which we redirected to stdout - - if s.returncode != 0 and print_ssh_output: - # extra leading newline is for spacing in wait_for_cluster_state() - print(textwrap.dedent("""\n - Warning: SSH connection error. (This could be temporary.) - Host: {h} - SSH return code: {r} - SSH output: {o} - """).format( - h=host, - r=s.returncode, - o=cmd_output.strip() - )) - - return s.returncode == 0 - - -def is_cluster_ssh_available(cluster_instances, opts): - """ - Check if SSH is available on all the instances in a cluster. - """ - for i in cluster_instances: - dns_name = get_dns_name(i, opts.private_ips) - if not is_ssh_available(host=dns_name, opts=opts): - return False - else: - return True - - -def wait_for_cluster_state(conn, opts, cluster_instances, cluster_state): - """ - Wait for all the instances in the cluster to reach a designated state. - - 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 cluster to enter '{s}' state.".format(s=cluster_state) - ) - sys.stdout.flush() - - start_time = datetime.now() - num_attempts = 0 - - while True: - time.sleep(5 * num_attempts) # seconds - - for i in cluster_instances: - i.update() - - max_batch = 100 - statuses = [] - for j in xrange(0, len(cluster_instances), max_batch): - batch = [i.id for i in cluster_instances[j:j + max_batch]] - statuses.extend(conn.get_all_instance_status(instance_ids=batch)) - - if cluster_state == 'ssh-ready': - if all(i.state == 'running' for i in cluster_instances) and \ - all(s.system_status.status == 'ok' for s in statuses) and \ - all(s.instance_status.status == 'ok' for s in statuses) 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") - - end_time = datetime.now() - print("Cluster is now in '{s}' state. Waited {t} seconds.".format( - s=cluster_state, - t=(end_time - start_time).seconds - )) - - -# Get number of local disks available for a given EC2 instance type. -def get_num_disks(instance_type): - # Source: http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/InstanceStorage.html - # Last Updated: 2015-06-19 - # For easy maintainability, please keep this manually-inputted dictionary sorted by key. - disks_by_instance = { - "c1.medium": 1, - "c1.xlarge": 4, - "c3.large": 2, - "c3.xlarge": 2, - "c3.2xlarge": 2, - "c3.4xlarge": 2, - "c3.8xlarge": 2, - "c4.large": 0, - "c4.xlarge": 0, - "c4.2xlarge": 0, - "c4.4xlarge": 0, - "c4.8xlarge": 0, - "cc1.4xlarge": 2, - "cc2.8xlarge": 4, - "cg1.4xlarge": 2, - "cr1.8xlarge": 2, - "d2.xlarge": 3, - "d2.2xlarge": 6, - "d2.4xlarge": 12, - "d2.8xlarge": 24, - "g2.2xlarge": 1, - "g2.8xlarge": 2, - "hi1.4xlarge": 2, - "hs1.8xlarge": 24, - "i2.xlarge": 1, - "i2.2xlarge": 2, - "i2.4xlarge": 4, - "i2.8xlarge": 8, - "m1.small": 1, - "m1.medium": 1, - "m1.large": 2, - "m1.xlarge": 4, - "m2.xlarge": 1, - "m2.2xlarge": 1, - "m2.4xlarge": 2, - "m3.medium": 1, - "m3.large": 1, - "m3.xlarge": 2, - "m3.2xlarge": 2, - "m4.large": 0, - "m4.xlarge": 0, - "m4.2xlarge": 0, - "m4.4xlarge": 0, - "m4.10xlarge": 0, - "r3.large": 1, - "r3.xlarge": 1, - "r3.2xlarge": 1, - "r3.4xlarge": 1, - "r3.8xlarge": 2, - "t1.micro": 0, - "t2.micro": 0, - "t2.small": 0, - "t2.medium": 0, - "t2.large": 0, - } - if instance_type in disks_by_instance: - return disks_by_instance[instance_type] - else: - print("WARNING: Don't know number of disks on instance type %s; assuming 1" - % instance_type, file=stderr) - return 1 - - -# Deploy the configuration file templates in a given local directory to -# a cluster, filling in any template parameters with information about the -# 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 = get_dns_name(master_nodes[0], opts.private_ips) - - num_disks = get_num_disks(opts.instance_type) - hdfs_data_dirs = "/mnt/ephemeral-hdfs/data" - mapred_local_dirs = "/mnt/hadoop/mrlocal" - spark_local_dirs = "/mnt/spark" - if num_disks > 1: - for i in range(2, num_disks + 1): - hdfs_data_dirs += ",/mnt%d/ephemeral-hdfs/data" % i - mapred_local_dirs += ",/mnt%d/hadoop/mrlocal" % i - spark_local_dirs += ",/mnt%d/spark" % i - - cluster_url = "%s:7077" % active_master - - if "." in opts.spark_version: - # Pre-built Spark deploy - spark_v = get_validate_spark_version(opts.spark_version, opts.spark_git_repo) - tachyon_v = get_tachyon_version(spark_v) - else: - # Spark-only custom deploy - spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version) - tachyon_v = "" - print("Deploying Spark via git hash; Tachyon won't be set up") - modules = filter(lambda x: x != "tachyon", modules) - - master_addresses = [get_dns_name(i, opts.private_ips) for i in master_nodes] - slave_addresses = [get_dns_name(i, opts.private_ips) for i in slave_nodes] - worker_instances_str = "%d" % opts.worker_instances if opts.worker_instances else "" - template_vars = { - "master_list": '\n'.join(master_addresses), - "active_master": active_master, - "slave_list": '\n'.join(slave_addresses), - "cluster_url": cluster_url, - "hdfs_data_dirs": hdfs_data_dirs, - "mapred_local_dirs": mapred_local_dirs, - "spark_local_dirs": spark_local_dirs, - "swap": str(opts.swap), - "modules": '\n'.join(modules), - "spark_version": spark_v, - "tachyon_version": tachyon_v, - "hadoop_major_version": opts.hadoop_major_version, - "spark_worker_instances": worker_instances_str, - "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() - for path, dirs, files in os.walk(root_dir): - if path.find(".svn") == -1: - dest_dir = os.path.join('/', path[len(root_dir):]) - local_dir = tmp_dir + dest_dir - if not os.path.exists(local_dir): - os.makedirs(local_dir) - for filename in files: - if filename[0] not in '#.~' and filename[-1] != '~': - dest_file = os.path.join(dest_dir, filename) - local_file = tmp_dir + dest_file - with open(os.path.join(path, filename)) as src: - with open(local_file, "w") as dest: - text = src.read() - for key in template_vars: - text = text.replace("{{" + key + "}}", template_vars[key]) - dest.write(text) - dest.close() - # rsync the whole directory over to the master machine - command = [ - 'rsync', '-rv', - '-e', stringify_command(ssh_command(opts)), - "%s/" % tmp_dir, - "%s@%s:/" % (opts.user, active_master) - ] - subprocess.check_call(command) - # Remove the temp directory we created above - shutil.rmtree(tmp_dir) - - -# Deploy a given local directory to a cluster, WITHOUT parameter substitution. -# Note that unlike deploy_files, this works for binary files. -# Also, it is up to the user to add (or not) the trailing slash in root_dir. -# Files are only deployed to the first master instance in the cluster. -# -# root_dir should be an absolute path. -def deploy_user_files(root_dir, opts, master_nodes): - active_master = get_dns_name(master_nodes[0], opts.private_ips) - command = [ - 'rsync', '-rv', - '-e', stringify_command(ssh_command(opts)), - "%s" % root_dir, - "%s@%s:/" % (opts.user, active_master) - ] - subprocess.check_call(command) - - -def stringify_command(parts): - if isinstance(parts, str): - return parts - else: - return ' '.join(map(pipes.quote, parts)) - - -def ssh_args(opts): - parts = ['-o', 'StrictHostKeyChecking=no'] - parts += ['-o', 'UserKnownHostsFile=/dev/null'] - if opts.identity_file is not None: - parts += ['-i', opts.identity_file] - return parts - - -def ssh_command(opts): - return ['ssh'] + ssh_args(opts) - - -# Run a command on a host through ssh, retrying up to five times -# and then throwing an exception if ssh continues to fail. -def ssh(host, opts, command): - tries = 0 - while True: - try: - return subprocess.check_call( - ssh_command(opts) + ['-t', '-t', '%s@%s' % (opts.user, host), - stringify_command(command)]) - except subprocess.CalledProcessError as e: - if tries > 5: - # If this was an ssh failure, provide the user with hints. - if e.returncode == 255: - raise UsageError( - "Failed to SSH to remote host {0}.\n" - "Please check that you have provided the correct --identity-file and " - "--key-pair parameters and try again.".format(host)) - else: - raise e - print("Error executing remote command, retrying after 30 seconds: {0}".format(e), - file=stderr) - time.sleep(30) - tries = tries + 1 - - -# Backported from Python 2.7 for compatiblity with 2.6 (See SPARK-1990) -def _check_output(*popenargs, **kwargs): - if 'stdout' in kwargs: - raise ValueError('stdout argument not allowed, it will be overridden.') - process = subprocess.Popen(stdout=subprocess.PIPE, *popenargs, **kwargs) - output, unused_err = process.communicate() - retcode = process.poll() - if retcode: - cmd = kwargs.get("args") - if cmd is None: - cmd = popenargs[0] - raise subprocess.CalledProcessError(retcode, cmd, output=output) - return output - - -def ssh_read(host, opts, command): - return _check_output( - ssh_command(opts) + ['%s@%s' % (opts.user, host), stringify_command(command)]) - - -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(arguments) - proc.stdin.close() - status = proc.wait() - if status == 0: - break - elif tries > 5: - raise RuntimeError("ssh_write failed with error %s" % proc.returncode) - else: - print("Error {0} while executing remote command, retrying after 30 seconds". - format(status), file=stderr) - time.sleep(30) - tries = tries + 1 - - -# Gets a list of zones to launch instances in -def get_zones(conn, opts): - if opts.zone == 'all': - zones = [z.name for z in conn.get_all_zones()] - else: - zones = [opts.zone] - return zones - - -# Gets the number of items in a partition -def get_partition(total, num_partitions, current_partitions): - num_slaves_this_zone = total // num_partitions - if (total % num_partitions) - current_partitions > 0: - num_slaves_this_zone += 1 - return num_slaves_this_zone - - -# Gets the IP address, taking into account the --private-ips flag -def get_ip_address(instance, private_ips=False): - ip = instance.ip_address if not private_ips else \ - instance.private_ip_address - return ip - - -# Gets the DNS name, taking into account the --private-ips flag -def get_dns_name(instance, private_ips=False): - dns = instance.public_dns_name if not private_ips else \ - instance.private_ip_address - if not dns: - raise UsageError("Failed to determine hostname of {0}.\n" - "Please check that you provided --private-ips if " - "necessary".format(instance)) - return dns - - -def real_main(): - (opts, action, cluster_name) = parse_args() - - # Input parameter validation - get_validate_spark_version(opts.spark_version, opts.spark_git_repo) - - 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 start up.", - DeprecationWarning - ) - - if opts.identity_file is not None: - if not os.path.exists(opts.identity_file): - print("ERROR: The identity file '{f}' doesn't exist.".format(f=opts.identity_file), - file=stderr) - sys.exit(1) - - file_mode = os.stat(opts.identity_file).st_mode - if not (file_mode & S_IRUSR) or not oct(file_mode)[-2:] == '00': - print("ERROR: The identity file must be accessible only by you.", file=stderr) - print('You can fix this with: chmod 400 "{f}"'.format(f=opts.identity_file), - file=stderr) - sys.exit(1) - - if opts.instance_type not in EC2_INSTANCE_TYPES: - print("Warning: Unrecognized EC2 instance type for instance-type: {t}".format( - t=opts.instance_type), file=stderr) - - if opts.master_instance_type != "": - if opts.master_instance_type not in EC2_INSTANCE_TYPES: - print("Warning: Unrecognized EC2 instance type for master-instance-type: {t}".format( - t=opts.master_instance_type), file=stderr) - # Since we try instance types even if we can't resolve them, we check if they resolve first - # and, if they do, see if they resolve to the same virtualization type. - if opts.instance_type in EC2_INSTANCE_TYPES and \ - opts.master_instance_type in EC2_INSTANCE_TYPES: - if EC2_INSTANCE_TYPES[opts.instance_type] != \ - EC2_INSTANCE_TYPES[opts.master_instance_type]: - print("Error: spark-ec2 currently does not support having a master and slaves " - "with different AMI virtualization types.", file=stderr) - print("master instance virtualization type: {t}".format( - t=EC2_INSTANCE_TYPES[opts.master_instance_type]), file=stderr) - print("slave instance virtualization type: {t}".format( - t=EC2_INSTANCE_TYPES[opts.instance_type]), file=stderr) - sys.exit(1) - - if opts.ebs_vol_num > 8: - print("ebs-vol-num cannot be greater than 8", file=stderr) - sys.exit(1) - - # Prevent breaking ami_prefix (/, .git and startswith checks) - # Prevent forks with non spark-ec2 names for now. - if opts.spark_ec2_git_repo.endswith("/") or \ - opts.spark_ec2_git_repo.endswith(".git") or \ - not opts.spark_ec2_git_repo.startswith("https://github.com") or \ - not opts.spark_ec2_git_repo.endswith("spark-ec2"): - print("spark-ec2-git-repo must be a github repo and it must not have a trailing / or .git. " - "Furthermore, we currently only support forks named spark-ec2.", file=stderr) - sys.exit(1) - - if not (opts.deploy_root_dir is None or - (os.path.isabs(opts.deploy_root_dir) and - os.path.isdir(opts.deploy_root_dir) and - os.path.exists(opts.deploy_root_dir))): - print("--deploy-root-dir must be an absolute path to a directory that exists " - "on the local file system", file=stderr) - sys.exit(1) - - try: - if opts.profile is None: - conn = ec2.connect_to_region(opts.region) - else: - conn = ec2.connect_to_region(opts.region, profile_name=opts.profile) - except Exception as e: - print((e), file=stderr) - sys.exit(1) - - # Select an AZ at random if it was not specified. - if opts.zone == "": - opts.zone = random.choice(conn.get_all_zones()).name - - if action == "launch": - if opts.slaves <= 0: - print("ERROR: You have to start at least 1 slave", file=sys.stderr) - sys.exit(1) - if opts.resume: - (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_state( - conn=conn, - opts=opts, - cluster_instances=(master_nodes + slave_nodes), - cluster_state='ssh-ready' - ) - setup_cluster(conn, master_nodes, slave_nodes, opts, True) - - elif action == "destroy": - (master_nodes, slave_nodes) = get_existing_cluster( - conn, opts, cluster_name, die_on_error=False) - - if any(master_nodes + slave_nodes): - print("The following instances will be terminated:") - for inst in master_nodes + slave_nodes: - print("> %s" % get_dns_name(inst, opts.private_ips)) - print("ALL DATA ON ALL NODES WILL BE LOST!!") - - msg = "Are you sure you want to destroy the cluster {c}? (y/N) ".format(c=cluster_name) - response = raw_input(msg) - if response == "y": - print("Terminating master...") - for inst in master_nodes: - inst.terminate() - print("Terminating slaves...") - for inst in slave_nodes: - inst.terminate() - - # Delete security groups as well - if opts.delete_groups: - group_names = [cluster_name + "-master", cluster_name + "-slaves"] - wait_for_cluster_state( - conn=conn, - opts=opts, - cluster_instances=(master_nodes + slave_nodes), - cluster_state='terminated' - ) - print("Deleting security groups (this will take some time)...") - attempt = 1 - while attempt <= 3: - print("Attempt %d" % attempt) - groups = [g for g in conn.get_all_security_groups() if g.name in group_names] - success = True - # Delete individual rules in all groups before deleting groups to - # remove dependencies between them - for group in groups: - print("Deleting rules in security group " + group.name) - for rule in group.rules: - for grant in rule.grants: - success &= group.revoke(ip_protocol=rule.ip_protocol, - from_port=rule.from_port, - to_port=rule.to_port, - src_group=grant) - - # Sleep for AWS eventual-consistency to catch up, and for instances - # to terminate - time.sleep(30) # Yes, it does have to be this long :-( - for group in groups: - try: - # It is needed to use group_id to make it work with VPC - conn.delete_security_group(group_id=group.id) - print("Deleted security group %s" % group.name) - except boto.exception.EC2ResponseError: - success = False - print("Failed to delete security group %s" % group.name) - - # Unfortunately, group.revoke() returns True even if a rule was not - # deleted, so this needs to be rerun if something fails - if success: - break - - attempt += 1 - - if not success: - print("Failed to delete all security groups after 3 tries.") - print("Try re-running in a few minutes.") - - elif action == "login": - (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - if not master_nodes[0].public_dns_name and not opts.private_ips: - print("Master has no public DNS name. Maybe you meant to specify --private-ips?") - else: - master = get_dns_name(master_nodes[0], opts.private_ips) - print("Logging into master " + master + "...") - proxy_opt = [] - if opts.proxy_port is not None: - proxy_opt = ['-D', opts.proxy_port] - 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) - if not master_nodes[0].public_dns_name and not opts.private_ips: - print("Master has no public DNS name. Maybe you meant to specify --private-ips?") - else: - print(get_dns_name(master_nodes[0], opts.private_ips)) - - elif action == "stop": - response = raw_input( - "Are you sure you want to stop the cluster " + - cluster_name + "?\nDATA ON EPHEMERAL DISKS WILL BE LOST, " + - "BUT THE CLUSTER WILL KEEP USING SPACE ON\n" + - "AMAZON EBS IF IT IS EBS-BACKED!!\n" + - "All data on spot-instance slaves will be lost.\n" + - "Stop cluster " + cluster_name + " (y/N): ") - if response == "y": - (master_nodes, slave_nodes) = get_existing_cluster( - conn, opts, cluster_name, die_on_error=False) - print("Stopping master...") - for inst in master_nodes: - if inst.state not in ["shutting-down", "terminated"]: - inst.stop() - print("Stopping slaves...") - for inst in slave_nodes: - if inst.state not in ["shutting-down", "terminated"]: - if inst.spot_instance_request_id: - inst.terminate() - else: - inst.stop() - - elif action == "start": - (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - print("Starting slaves...") - for inst in slave_nodes: - if inst.state not in ["shutting-down", "terminated"]: - inst.start() - print("Starting master...") - for inst in master_nodes: - if inst.state not in ["shutting-down", "terminated"]: - inst.start() - wait_for_cluster_state( - conn=conn, - opts=opts, - cluster_instances=(master_nodes + slave_nodes), - cluster_state='ssh-ready' - ) - - # Determine types of running instances - existing_master_type = master_nodes[0].instance_type - existing_slave_type = slave_nodes[0].instance_type - # Setting opts.master_instance_type to the empty string indicates we - # have the same instance type for the master and the slaves - if existing_master_type == existing_slave_type: - existing_master_type = "" - opts.master_instance_type = existing_master_type - opts.instance_type = existing_slave_type - - setup_cluster(conn, master_nodes, slave_nodes, opts, False) - - else: - print("Invalid action: %s" % action, file=stderr) - sys.exit(1) - - -def main(): - try: - real_main() - except UsageError as e: - print("\nError:\n", e, file=stderr) - sys.exit(1) - - -if __name__ == "__main__": - logging.basicConfig() - main() diff --git a/make-distribution.sh b/make-distribution.sh index a38fd8df17206..327659298e4d8 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -212,7 +212,6 @@ cp "$SPARK_HOME/README.md" "$DISTDIR" cp -r "$SPARK_HOME/bin" "$DISTDIR" cp -r "$SPARK_HOME/python" "$DISTDIR" cp -r "$SPARK_HOME/sbin" "$DISTDIR" -cp -r "$SPARK_HOME/ec2" "$DISTDIR" # Copy SparkR if it exists if [ -d "$SPARK_HOME"/R/lib/SparkR ]; then mkdir -p "$DISTDIR"/R/lib From b78e028e37193a4e27b012f0b3c8343d850c5674 Mon Sep 17 00:00:00 2001 From: Jacek Laskowski Date: Sun, 10 Jan 2016 10:36:01 +0000 Subject: [PATCH 052/110] =?UTF-8?q?[SPARK-12736][CORE][DEPLOY]=20Standalon?= =?UTF-8?q?e=20Master=20cannot=20be=20started=20due=20t=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …o NoClassDefFoundError: org/spark-project/guava/collect/Maps /cc srowen rxin Author: Jacek Laskowski Closes #10674 from jaceklaskowski/SPARK-12736. --- network/common/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/network/common/pom.xml b/network/common/pom.xml index 92ca0046d4f53..eda2b7307088f 100644 --- a/network/common/pom.xml +++ b/network/common/pom.xml @@ -55,6 +55,7 @@ com.google.guava guava + compile From e5904bb5e7d83b3731b312c40f7904c0511019f5 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 10 Jan 2016 12:38:57 -0800 Subject: [PATCH 053/110] [SPARK-12692][BUILD][MLLIB] Scala style: Fix the style violation (Space before "," or ":") Fix the style violation (space before , and :). This PR is a followup for #10643. Author: Kousuke Saruta Closes #10684 from sarutak/SPARK-12692-followup-mllib. --- .../spark/examples/mllib/IsotonicRegressionExample.scala | 2 +- .../org/apache/spark/examples/mllib/NaiveBayesExample.scala | 2 +- .../spark/examples/mllib/RegressionMetricsExample.scala | 2 +- .../org/apache/spark/ml/classification/OneVsRest.scala | 4 ++-- .../main/scala/org/apache/spark/ml/feature/RFormula.scala | 2 +- .../scala/org/apache/spark/ml/feature/VectorAssembler.scala | 2 +- .../scala/org/apache/spark/ml/tree/impl/RandomForest.scala | 4 ++-- .../spark/mllib/clustering/GaussianMixtureModel.scala | 2 +- .../main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala | 2 +- .../main/scala/org/apache/spark/mllib/linalg/Matrices.scala | 2 +- .../pmml/export/BinaryClassificationPMMLModelExport.scala | 6 +++--- .../spark/mllib/pmml/export/KMeansPMMLModelExport.scala | 4 ++-- .../scala/org/apache/spark/mllib/tree/DecisionTree.scala | 2 +- .../main/scala/org/apache/spark/mllib/tree/model/Node.scala | 2 +- .../org/apache/spark/mllib/util/LinearDataGenerator.scala | 2 +- .../org/apache/spark/mllib/classification/SVMSuite.scala | 2 +- .../org/apache/spark/mllib/stat/StreamingTestSuite.scala | 2 +- 17 files changed, 22 insertions(+), 22 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/IsotonicRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/IsotonicRegressionExample.scala index 3834ea807acbf..c4336639d7c0b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/IsotonicRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/IsotonicRegressionExample.scala @@ -25,7 +25,7 @@ import org.apache.spark.mllib.regression.{IsotonicRegression, IsotonicRegression object IsotonicRegressionExample { - def main(args: Array[String]) : Unit = { + def main(args: Array[String]): Unit = { val conf = new SparkConf().setAppName("IsotonicRegressionExample") val sc = new SparkContext(conf) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/NaiveBayesExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/NaiveBayesExample.scala index 8bae1b9d1832d..0187ad603a654 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/NaiveBayesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/NaiveBayesExample.scala @@ -27,7 +27,7 @@ import org.apache.spark.mllib.regression.LabeledPoint object NaiveBayesExample { - def main(args: Array[String]) : Unit = { + def main(args: Array[String]): Unit = { val conf = new SparkConf().setAppName("NaiveBayesExample") val sc = new SparkContext(conf) // $example on$ diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala index ace16ff1ea225..add634c957b40 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala @@ -27,7 +27,7 @@ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.SQLContext object RegressionMetricsExample { - def main(args: Array[String]) : Unit = { + def main(args: Array[String]): Unit = { val conf = new SparkConf().setAppName("RegressionMetricsExample") val sc = new SparkContext(conf) val sqlContext = new SQLContext(sc) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index 08a51109d6c62..c41a611f1cc60 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -113,13 +113,13 @@ final class OneVsRestModel private[ml] ( val updateUDF = udf { (predictions: Map[Int, Double], prediction: Vector) => predictions + ((index, prediction(1))) } - val transformedDataset = model.transform(df).select(columns : _*) + val transformedDataset = model.transform(df).select(columns: _*) val updatedDataset = transformedDataset .withColumn(tmpColName, updateUDF(col(accColName), col(rawPredictionCol))) val newColumns = origCols ++ List(col(tmpColName)) // switch out the intermediate column with the accumulator column - updatedDataset.select(newColumns : _*).withColumnRenamed(tmpColName, accColName) + updatedDataset.select(newColumns: _*).withColumnRenamed(tmpColName, accColName) } if (handlePersistence) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index f9952434d2982..6cc9d025445c0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -238,7 +238,7 @@ private class ColumnPruner(columnsToPrune: Set[String]) extends Transformer { override def transform(dataset: DataFrame): DataFrame = { val columnsToKeep = dataset.columns.filter(!columnsToPrune.contains(_)) - dataset.select(columnsToKeep.map(dataset.col) : _*) + dataset.select(columnsToKeep.map(dataset.col): _*) } override def transformSchema(schema: StructType): StructType = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala index 0b215659b3672..716bc63e00995 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -102,7 +102,7 @@ class VectorAssembler(override val uid: String) } } - dataset.select(col("*"), assembleFunc(struct(args : _*)).as($(outputCol), metadata)) + dataset.select(col("*"), assembleFunc(struct(args: _*)).as($(outputCol), metadata)) } override def transformSchema(schema: StructType): StructType = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index 6e87302c7779b..d3376a7dff938 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -474,7 +474,7 @@ private[ml] object RandomForest extends Logging { val nodeToFeatures = getNodeToFeatures(treeToNodeToIndexInfo) val nodeToFeaturesBc = input.sparkContext.broadcast(nodeToFeatures) - val partitionAggregates : RDD[(Int, DTStatsAggregator)] = if (nodeIdCache.nonEmpty) { + 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 @@ -825,7 +825,7 @@ private[ml] object RandomForest extends Logging { protected[tree] def findSplits( input: RDD[LabeledPoint], metadata: DecisionTreeMetadata, - seed : Long): Array[Array[Split]] = { + seed: Long): Array[Array[Split]] = { logDebug("isMulticlass = " + metadata.isMulticlass) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index 5c9bc62cb09bb..16bc45bcb627f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -177,7 +177,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { } @Since("1.4.0") - override def load(sc: SparkContext, path: String) : GaussianMixtureModel = { + override def load(sc: SparkContext, path: String): GaussianMixtureModel = { val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) implicit val formats = DefaultFormats val k = (metadata \ "k").extract[Int] diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index 5273ed4d76650..ffae0e7ed0ca4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -134,7 +134,7 @@ object FPGrowthModel extends Loader[FPGrowthModel[_]] { loadImpl(freqItemsets, sample) } - def loadImpl[Item : ClassTag](freqItemsets: DataFrame, sample: Item): FPGrowthModel[Item] = { + def loadImpl[Item: ClassTag](freqItemsets: DataFrame, sample: Item): FPGrowthModel[Item] = { val freqItemsetsRDD = freqItemsets.select("items", "freq").map { x => val items = x.getAs[Seq[Item]](0).toArray val freq = x.getLong(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 d7a74db0b1fd8..b08da4fb55034 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 @@ -279,7 +279,7 @@ class DenseMatrix @Since("1.3.0") ( } override def hashCode: Int = { - com.google.common.base.Objects.hashCode(numRows : Integer, numCols: Integer, toArray) + com.google.common.base.Objects.hashCode(numRows: Integer, numCols: Integer, toArray) } private[mllib] def toBreeze: BM[Double] = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala index 7abb1bf7ce967..a8c32f72bfdeb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala @@ -27,9 +27,9 @@ import org.apache.spark.mllib.regression.GeneralizedLinearModel * PMML Model Export for GeneralizedLinearModel class with binary ClassificationModel */ private[mllib] class BinaryClassificationPMMLModelExport( - model : GeneralizedLinearModel, - description : String, - normalizationMethod : RegressionNormalizationMethodType, + model: GeneralizedLinearModel, + description: String, + normalizationMethod: RegressionNormalizationMethodType, threshold: Double) extends PMMLModelExport { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala index b5b824bb9c9b6..255c6140e5410 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala @@ -26,14 +26,14 @@ import org.apache.spark.mllib.clustering.KMeansModel /** * PMML Model Export for KMeansModel class */ -private[mllib] class KMeansPMMLModelExport(model : KMeansModel) extends PMMLModelExport{ +private[mllib] class KMeansPMMLModelExport(model: KMeansModel) extends PMMLModelExport{ populateKMeansPMML(model) /** * Export the input KMeansModel model to PMML format. */ - private def populateKMeansPMML(model : KMeansModel): Unit = { + private def populateKMeansPMML(model: KMeansModel): Unit = { pmml.getHeader.setDescription("k-means clustering") if (model.clusterCenters.length > 0) { 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 af1f7e74c004d..c73774fcd8c46 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 @@ -600,7 +600,7 @@ object DecisionTree extends Serializable with Logging { val nodeToFeatures = getNodeToFeatures(treeToNodeToIndexInfo) val nodeToFeaturesBc = input.sparkContext.broadcast(nodeToFeatures) - val partitionAggregates : RDD[(Int, DTStatsAggregator)] = if (nodeIdCache.nonEmpty) { + 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 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 66f0908c1250f..b373c2de3ea96 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 @@ -83,7 +83,7 @@ class Node @Since("1.2.0") ( * @return predicted value */ @Since("1.1.0") - def predict(features: Vector) : Double = { + def predict(features: Vector): Double = { if (isLeaf) { predict.predict } else { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 094528e2ece06..240781bcd335b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -175,7 +175,7 @@ object LinearDataGenerator { nfeatures: Int, eps: Double, nparts: Int = 2, - intercept: Double = 0.0) : RDD[LabeledPoint] = { + intercept: Double = 0.0): RDD[LabeledPoint] = { val random = new Random(42) // Random values distributed uniformly in [-0.5, 0.5] val w = Array.fill(nfeatures)(random.nextDouble() - 0.5) 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 ee3c85d09a463..1a47344b68937 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 @@ -45,7 +45,7 @@ object SVMSuite { nPoints: Int, seed: Int): Seq[LabeledPoint] = { val rnd = new Random(seed) - val weightsMat = new DoubleMatrix(1, weights.length, weights : _*) + val weightsMat = new DoubleMatrix(1, weights.length, weights: _*) val x = Array.fill[Array[Double]](nPoints)( Array.fill[Double](weights.length)(rnd.nextDouble() * 2.0 - 1.0)) val y = x.map { xi => diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala index 1142102bb040e..50441816ece3e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.random.XORShiftRandom class StreamingTestSuite extends SparkFunSuite with TestSuiteBase { - override def maxWaitTimeMillis : Int = 30000 + override def maxWaitTimeMillis: Int = 30000 test("accuracy for null hypothesis using welch t-test") { // set parameters From 3119206b7188c23055621dfeaf6874f21c711a82 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 10 Jan 2016 15:41:22 -0800 Subject: [PATCH 054/110] [SPARK-12692][BUILD][GRAPHX] Scala style: Fix the style violation (Space before "," or ":") Fix the style violation (space before `,` and `:`). This PR is a followup for #10643. Author: Kousuke Saruta Closes #10683 from sarutak/SPARK-12692-followup-graphx. --- graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala | 5 ++--- .../org/apache/spark/graphx/impl/ReplicatedVertexView.scala | 4 ++-- .../apache/spark/graphx/impl/ShippableVertexPartition.scala | 4 ++-- .../apache/spark/graphx/impl/VertexPartitionBaseOps.scala | 2 +- .../main/scala/org/apache/spark/graphx/lib/PageRank.scala | 2 +- 5 files changed, 8 insertions(+), 9 deletions(-) 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 fc36e12dd2aed..d048fb5d561f3 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -21,7 +21,6 @@ import scala.reflect.ClassTag import scala.util.Random import org.apache.spark.SparkException -import org.apache.spark.SparkContext._ import org.apache.spark.graphx.lib._ import org.apache.spark.rdd.RDD @@ -379,7 +378,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * @see [[org.apache.spark.graphx.lib.PageRank$#runUntilConvergenceWithOptions]] */ def personalizedPageRank(src: VertexId, tol: Double, - resetProb: Double = 0.15) : Graph[Double, Double] = { + resetProb: Double = 0.15): Graph[Double, Double] = { PageRank.runUntilConvergenceWithOptions(graph, tol, resetProb, Some(src)) } @@ -392,7 +391,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * @see [[org.apache.spark.graphx.lib.PageRank$#runWithOptions]] */ def staticPersonalizedPageRank(src: VertexId, numIter: Int, - resetProb: Double = 0.15) : Graph[Double, Double] = { + resetProb: Double = 0.15): Graph[Double, Double] = { PageRank.runWithOptions(graph, numIter, resetProb, Some(src)) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala index f79f9c7ec448f..b4bec7cba5207 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala @@ -41,8 +41,8 @@ class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( * shipping level. */ def withEdges[VD2: ClassTag, ED2: ClassTag]( - edges_ : EdgeRDDImpl[ED2, VD2]): ReplicatedVertexView[VD2, ED2] = { - new ReplicatedVertexView(edges_, hasSrcId, hasDstId) + _edges: EdgeRDDImpl[ED2, VD2]): ReplicatedVertexView[VD2, ED2] = { + new ReplicatedVertexView(_edges, hasSrcId, hasDstId) } /** 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 3f203c4eca485..96d807f9f9ceb 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 @@ -102,8 +102,8 @@ class ShippableVertexPartition[VD: ClassTag]( extends VertexPartitionBase[VD] { /** Return a new ShippableVertexPartition with the specified routing table. */ - def withRoutingTable(routingTable_ : RoutingTablePartition): ShippableVertexPartition[VD] = { - new ShippableVertexPartition(index, values, mask, routingTable_) + def withRoutingTable(_routingTable: RoutingTablePartition): ShippableVertexPartition[VD] = { + new ShippableVertexPartition(index, values, mask, _routingTable) } /** diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index f508b483a2f1b..7c680dcb99cd2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.collection.BitSet * example, [[VertexPartition.VertexPartitionOpsConstructor]]). */ private[graphx] abstract class VertexPartitionBaseOps - [VD: ClassTag, Self[X] <: VertexPartitionBase[X] : VertexPartitionBaseOpsConstructor] + [VD: ClassTag, Self[X] <: VertexPartitionBase[X]: VertexPartitionBaseOpsConstructor] (self: Self[VD]) extends Serializable with Logging { 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 35b26c998e1d9..46faad2e68c50 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 @@ -138,7 +138,7 @@ object PageRank extends Logging { // edge partitions. prevRankGraph = rankGraph val rPrb = if (personalized) { - (src: VertexId , id: VertexId) => resetProb * delta(src, id) + (src: VertexId, id: VertexId) => resetProb * delta(src, id) } else { (src: VertexId, id: VertexId) => resetProb } From 3ab0138b0fe0f9208b4b476855294a7c729583b7 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 10 Jan 2016 19:59:01 -0800 Subject: [PATCH 055/110] [SPARK-12734][BUILD] Fix Netty exclusion and use Maven Enforcer to prevent future bugs Netty classes are published under multiple artifacts with different names, so our build needs to exclude the `io.netty:netty` and `org.jboss.netty:netty` versions of the Netty artifact. However, our existing exclusions were incomplete, leading to situations where duplicate Netty classes would wind up on the classpath and cause compile errors (or worse). This patch fixes the exclusion issue by adding more exclusions and uses Maven Enforcer's [banned dependencies](https://maven.apache.org/enforcer/enforcer-rules/bannedDependencies.html) rule to prevent these classes from accidentally being reintroduced. I also updated `dev/test-dependencies.sh` to run `mvn validate` so that the enforcer rules can run as part of pull request builds. /cc rxin srowen pwendell. I'd like to backport at least the exclusion portion of this fix to `branch-1.5` in order to fix the documentation publishing job, which fails nondeterministically due to incompatible versions of Netty classes taking precedence on the compile-time classpath. Author: Josh Rosen Author: Josh Rosen Closes #10672 from JoshRosen/enforce-netty-exclusions. --- dev/deps/spark-deps-hadoop-2.2 | 1 - dev/deps/spark-deps-hadoop-2.3 | 1 - dev/deps/spark-deps-hadoop-2.4 | 1 - dev/deps/spark-deps-hadoop-2.6 | 1 - dev/test-dependencies.sh | 17 +++------- examples/pom.xml | 4 +++ pom.xml | 57 +++++++++++++++++++++++++++++++++- 7 files changed, 64 insertions(+), 18 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index e4373f79f7922..13d1b0e950480 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -142,7 +142,6 @@ metrics-graphite-3.1.2.jar metrics-json-3.1.2.jar metrics-jvm-3.1.2.jar minlog-1.2.jar -netty-3.8.0.Final.jar netty-all-4.0.29.Final.jar objenesis-1.2.jar opencsv-2.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 7478181406d07..d7deaa0a24541 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -133,7 +133,6 @@ metrics-json-3.1.2.jar metrics-jvm-3.1.2.jar minlog-1.2.jar mx4j-3.0.2.jar -netty-3.8.0.Final.jar netty-all-4.0.29.Final.jar objenesis-1.2.jar opencsv-2.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index faffb8bf398a5..7ad2212ed5ae7 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -134,7 +134,6 @@ metrics-json-3.1.2.jar metrics-jvm-3.1.2.jar minlog-1.2.jar mx4j-3.0.2.jar -netty-3.8.0.Final.jar netty-all-4.0.29.Final.jar objenesis-1.2.jar opencsv-2.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index e703c7acd3876..7f8518927aec4 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -140,7 +140,6 @@ metrics-json-3.1.2.jar metrics-jvm-3.1.2.jar minlog-1.2.jar mx4j-3.0.2.jar -netty-3.8.0.Final.jar netty-all-4.0.29.Final.jar objenesis-1.2.jar opencsv-2.3.jar diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index 424ce6ad7663c..def87aa4087e3 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -70,19 +70,10 @@ $MVN -q versions:set -DnewVersion=$TEMP_VERSION -DgenerateBackupPoms=false > /de # Generate manifests for each Hadoop profile: for HADOOP_PROFILE in "${HADOOP_PROFILES[@]}"; do echo "Performing Maven install for $HADOOP_PROFILE" - $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE jar:jar install:install -q \ - -pl '!assembly' \ - -pl '!examples' \ - -pl '!external/flume-assembly' \ - -pl '!external/kafka-assembly' \ - -pl '!external/twitter' \ - -pl '!external/flume' \ - -pl '!external/mqtt' \ - -pl '!external/mqtt-assembly' \ - -pl '!external/zeromq' \ - -pl '!external/kafka' \ - -pl '!tags' \ - -DskipTests + $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE jar:jar jar:test-jar install:install -q + + echo "Performing Maven validate for $HADOOP_PROFILE" + $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE validate -q echo "Generating dependency manifest for $HADOOP_PROFILE" mkdir -p dev/pr-deps diff --git a/examples/pom.xml b/examples/pom.xml index 1a0d5e5854642..6013085b10e84 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -111,6 +111,10 @@ org.jruby jruby-complete + + io.netty + netty +
    diff --git a/pom.xml b/pom.xml index 0eac212754320..cbed36c1eac16 100644 --- a/pom.xml +++ b/pom.xml @@ -519,6 +519,12 @@ ${akka.group} akka-remote_${scala.binary.version} ${akka.version} + + + io.netty + netty + + ${akka.group} @@ -762,6 +768,10 @@ org.jboss.netty netty + + io.netty + netty + @@ -822,6 +832,10 @@ junit junit + + io.netty + netty + @@ -922,6 +936,10 @@ org.jboss.netty netty + + io.netty + netty + commons-logging commons-logging @@ -946,6 +964,10 @@ org.jboss.netty netty + + io.netty + netty + javax.servlet servlet-api @@ -975,6 +997,10 @@ org.jboss.netty netty + + io.netty + netty + javax.servlet servlet-api @@ -1003,6 +1029,10 @@ org.jboss.netty netty + + io.netty + netty + javax.servlet servlet-api @@ -1031,6 +1061,10 @@ org.jboss.netty netty + + io.netty + netty + javax.servlet servlet-api @@ -1046,6 +1080,16 @@ zookeeper ${zookeeper.version} ${hadoop.deps.scope} + + + org.jboss.netty + netty + + + io.netty + netty + + org.codehaus.jackson @@ -1771,7 +1815,7 @@ org.apache.maven.plugins maven-enforcer-plugin - 1.4 + 1.4.1 enforce-versions @@ -1786,6 +1830,17 @@ ${java.version} + + + io.netty:netty + org.jboss.netty + + + + io.netty:netty:3.4.0.Final:*:test + + true + From 6439a82503e900ae2e5c3cda5d10ac20dfd6e53f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Sun, 10 Jan 2016 20:04:50 -0800 Subject: [PATCH 056/110] [SPARK-3873][BUILD] Enable import ordering error checking. Turn import ordering violations into build errors, plus a few adjustments to account for how the checker behaves. I'm a little on the fence about whether the existing code is right, but it's easier to appease the checker than to discuss what's the more correct order here. Plus a few fixes to imports that cropped in since my recent cleanups. Author: Marcelo Vanzin Closes #10612 from vanzin/SPARK-3873-enable. --- .../streaming/KinesisWordCountASL.scala | 2 +- .../kinesis/KinesisInputDStream.scala | 2 +- .../streaming/kinesis/KinesisReceiver.scala | 2 +- .../streaming/kinesis/KinesisUtils.scala | 2 +- .../kinesis/KinesisBackedBlockRDDSuite.scala | 2 +- .../kinesis/KinesisCheckpointerSuite.scala | 4 ++-- .../kinesis/KinesisReceiverSuite.scala | 2 +- .../kinesis/KinesisStreamSuite.scala | 4 ++-- .../org/apache/spark/ml/tree/treeParams.scala | 2 +- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 7 +++---- .../spark/mllib/tree/DecisionTree.scala | 2 +- .../mllib/tree/GradientBoostedTrees.scala | 2 +- .../spark/mllib/tree/RandomForest.scala | 2 +- scalastyle-config.xml | 21 +++++++++---------- .../spark/sql/catalyst/CatalystQl.scala | 2 +- .../sql/catalyst/expressions/literals.scala | 2 +- .../apache/spark/sql/types/StructType.scala | 3 +-- .../apache/spark/sql/execution/SparkQl.scala | 2 +- .../apache/spark/sql/execution/Window.scala | 2 +- .../datasources/WriterContainer.scala | 2 +- .../sql/execution/datasources/bucket.scala | 3 ++- .../datasources/parquet/ParquetRelation.scala | 10 ++++----- .../sql/execution/datasources/rules.scala | 2 +- .../parquet/ParquetReadBenchmark.scala | 2 +- .../org/apache/spark/sql/hive/HiveQl.scala | 5 +++-- .../apache/spark/sql/hive/SQLBuilder.scala | 2 +- .../org/apache/spark/sql/hive/hiveUDFs.scala | 8 +++---- .../apache/spark/sql/hive/test/TestHive.scala | 2 +- .../spark/sql/hive/ErrorPositionSuite.scala | 2 +- .../sql/hive/LogicalPlanToSQLSuite.scala | 2 +- .../spark/sql/hive/SQLBuilderTest.scala | 2 +- .../hive/execution/HiveComparisonTest.scala | 4 ++-- .../sql/sources/BucketedWriteSuite.scala | 4 ++-- .../apache/spark/streaming/Checkpoint.scala | 2 +- .../spark/streaming/util/StateMap.scala | 2 +- .../spark/streaming/StateMapSuite.scala | 5 ++--- 36 files changed, 62 insertions(+), 64 deletions(-) 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 de749626ec09c..6a73bc0e30d05 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 @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import scala.util.Random -import com.amazonaws.auth.{DefaultAWSCredentialsProviderChain, BasicAWSCredentials} +import com.amazonaws.auth.{BasicAWSCredentials, DefaultAWSCredentialsProviderChain} import com.amazonaws.regions.RegionUtils import com.amazonaws.services.kinesis.AmazonKinesisClient import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala index 3321c7527edb4..5223c81a8e0e0 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala @@ -24,10 +24,10 @@ import com.amazonaws.services.kinesis.model.Record import org.apache.spark.rdd.RDD import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.streaming.{Duration, StreamingContext, Time} import org.apache.spark.streaming.dstream.ReceiverInputDStream import org.apache.spark.streaming.receiver.Receiver import org.apache.spark.streaming.scheduler.ReceivedBlockInfo -import org.apache.spark.streaming.{Duration, StreamingContext, Time} private[kinesis] class KinesisInputDStream[T: ClassTag]( _ssc: StreamingContext, 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 index abb9b6cd32f1c..48ee2a959786b 100644 --- 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 @@ -24,7 +24,7 @@ import scala.collection.mutable import scala.util.control.NonFatal import com.amazonaws.auth.{AWSCredentials, AWSCredentialsProvider, DefaultAWSCredentialsProviderChain} -import com.amazonaws.services.kinesis.clientlibrary.interfaces.{IRecordProcessorCheckpointer, IRecordProcessor, IRecordProcessorFactory} +import com.amazonaws.services.kinesis.clientlibrary.interfaces.{IRecordProcessor, IRecordProcessorCheckpointer, IRecordProcessorFactory} import com.amazonaws.services.kinesis.clientlibrary.lib.worker.{InitialPositionInStream, KinesisClientLibConfiguration, Worker} import com.amazonaws.services.kinesis.model.Record 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 2de6195716e5c..15ac588b82587 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 @@ -24,9 +24,9 @@ import com.amazonaws.services.kinesis.model.Record import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Duration, StreamingContext} import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} import org.apache.spark.streaming.dstream.ReceiverInputDStream -import org.apache.spark.streaming.{Duration, StreamingContext} object KinesisUtils { /** diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala index d85b4cda8ce98..e6f504c4e54dd 100644 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala +++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala @@ -19,8 +19,8 @@ package org.apache.spark.streaming.kinesis import org.scalatest.BeforeAndAfterAll -import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBlockId} import org.apache.spark.{SparkConf, SparkContext, SparkException} +import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBlockId} abstract class KinesisBackedBlockRDDTests(aggregateTestData: Boolean) extends KinesisFunSuite with BeforeAndAfterAll { diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala index 645e64a0bc3a0..e1499a8220991 100644 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala +++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.kinesis -import java.util.concurrent.{TimeoutException, ExecutorService} +import java.util.concurrent.{ExecutorService, TimeoutException} import scala.concurrent.{Await, ExecutionContext, Future} import scala.concurrent.duration._ @@ -28,7 +28,7 @@ import org.mockito.Matchers._ import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer -import org.scalatest.{PrivateMethodTester, BeforeAndAfterEach} +import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually._ import org.scalatest.mock.MockitoSugar 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 index e5c70db554a27..fd15b6ccdc889 100644 --- 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 @@ -27,8 +27,8 @@ import com.amazonaws.services.kinesis.model.Record import org.mockito.Matchers._ import org.mockito.Matchers.{eq => meq} import org.mockito.Mockito._ -import org.scalatest.mock.MockitoSugar import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.mock.MockitoSugar import org.apache.spark.streaming.{Duration, TestSuiteBase} import org.apache.spark.util.Utils diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala index 78263f9dca65c..ee6a5f0390d04 100644 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala +++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala @@ -25,10 +25,11 @@ import scala.util.Random import com.amazonaws.regions.RegionUtils import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream import com.amazonaws.services.kinesis.model.Record +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} import org.scalatest.Matchers._ import org.scalatest.concurrent.Eventually -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.storage.{StorageLevel, StreamBlockId} @@ -38,7 +39,6 @@ import org.apache.spark.streaming.kinesis.KinesisTestUtils._ import org.apache.spark.streaming.receiver.BlockManagerBasedStoreResult import org.apache.spark.streaming.scheduler.ReceivedBlockInfo import org.apache.spark.util.Utils -import org.apache.spark.{SparkConf, SparkContext} abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFunSuite with Eventually with BeforeAndAfter with BeforeAndAfterAll { diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala index 7443097492d82..7a651a37ac77e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala @@ -24,7 +24,7 @@ import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, BoostingStrategy => OldBoostingStrategy, Strategy => OldStrategy} import org.apache.spark.mllib.tree.impurity.{Entropy => OldEntropy, Gini => OldGini, Impurity => OldImpurity, Variance => OldVariance} import org.apache.spark.mllib.tree.loss.{Loss => OldLoss} -import org.apache.spark.sql.types.{DoubleType, DataType, StructType} +import org.apache.spark.sql.types.{DataType, DoubleType, StructType} /** * Parameters for Decision Tree-based algorithms. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index ffae0e7ed0ca4..1250bc1a07cb4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -17,8 +17,8 @@ package org.apache.spark.mllib.fpm -import java.lang.{Iterable => JavaIterable} import java.{util => ju} +import java.lang.{Iterable => JavaIterable} import scala.collection.JavaConverters._ import scala.collection.mutable @@ -29,16 +29,15 @@ import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods.{compact, render} -import org.apache.spark.{HashPartitioner, Logging, Partitioner, SparkException} +import org.apache.spark.{HashPartitioner, Logging, Partitioner, SparkContext, SparkException} import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.mllib.fpm.FPGrowth._ import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext -import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel 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 c73774fcd8c46..07ba0d8ccb2a8 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 @@ -25,10 +25,10 @@ import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD 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._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ +import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impl._ import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.model._ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index 729a211574822..1b71256c585bd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -22,8 +22,8 @@ import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.impl.PeriodicRDDCheckpointer import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.configuration.BoostingStrategy import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.BoostingStrategy import org.apache.spark.mllib.tree.impl.TimeTracker import org.apache.spark.mllib.tree.impurity.Variance import org.apache.spark.mllib.tree.model.{DecisionTreeModel, GradientBoostedTreesModel} 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 a684cdd18c2fc..570a76f960796 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,9 +26,9 @@ import org.apache.spark.Logging import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ +import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impl.{BaggedPoint, DecisionTreeMetadata, NodeIdCache, TimeTracker, TreePoint} import org.apache.spark.mllib.tree.impurity.Impurities diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 9714c46fe99a0..2439a1f715aba 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -187,6 +187,16 @@ This file is divided into 3 sections: scala.collection.JavaConverters._ and use .asScala / .asJava methods + + + java,scala,3rdParty,spark + javax?\..* + scala\..* + (?!org\.apache\.spark\.).* + org\.apache\.spark\..* + + + @@ -207,17 +217,6 @@ This file is divided into 3 sections: - - - - java,scala,3rdParty,spark - javax?\..* - scala\..* - (?!org\.apache\.spark\.).* - org\.apache\.spark\..* - - - diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala index 1eda4a9a97644..2e3cc0bfde7c7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala @@ -22,10 +22,10 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.Count +import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.CurrentOrigin -import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.util.random.RandomSampler 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 0eb915fdc1691..17351ef0685a9 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 @@ -21,9 +21,9 @@ import java.sql.{Date, Timestamp} import org.json4s.JsonAST._ +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 9b5c86a8984be..3bd733fa2d26c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -25,8 +25,7 @@ import org.json4s.JsonDSL._ import org.apache.spark.SparkException import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering} -import org.apache.spark.sql.catalyst.util.{LegacyTypeStringParser, DataTypeParser} - +import org.apache.spark.sql.catalyst.util.{DataTypeParser, LegacyTypeStringParser} /** * :: DeveloperApi :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala index a322688a259e2..f3e89ef4a71f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.sql.execution +import org.apache.spark.sql.catalyst.{CatalystQl, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} -import org.apache.spark.sql.catalyst.{CatalystQl, TableIdentifier} private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends CatalystQl(conf) { /** Check if a command should not be explained. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala index be885397a7d40..168b5ab0316d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala @@ -22,6 +22,7 @@ import java.util import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ @@ -29,7 +30,6 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.types.IntegerType import org.apache.spark.util.collection.unsafe.sort.{UnsafeExternalSorter, UnsafeSorterIterator} -import org.apache.spark.{SparkEnv, TaskContext} /** * This class calculates and outputs (windowed) aggregates over the rows in a single (sorted) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index 4f8524f4b967c..40ecdb8e4403e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.UnsafeKVExternalSorter import org.apache.spark.sql.sources.{HadoopFsRelation, OutputWriter, OutputWriterFactory} -import org.apache.spark.sql.types.{IntegerType, StructType, StringType} +import org.apache.spark.sql.types.{IntegerType, StringType, StructType} import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala index 82287c8967134..9976829638d70 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala @@ -18,8 +18,9 @@ package org.apache.spark.sql.execution.datasources import org.apache.hadoop.mapreduce.TaskAttemptContext + import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.sources.{OutputWriter, OutputWriterFactory, HadoopFsRelationProvider, HadoopFsRelation} +import org.apache.spark.sql.sources.{HadoopFsRelation, HadoopFsRelationProvider, OutputWriter, OutputWriterFactory} import org.apache.spark.sql.types.StructType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index ca8d010090401..7754edc803d10 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.execution.datasources.parquet import java.net.URI -import java.util.logging.{Logger => JLogger} import java.util.{List => JList} +import java.util.logging.{Logger => JLogger} import scala.collection.JavaConverters._ import scala.collection.mutable @@ -32,24 +32,24 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.task.JobContextImpl +import org.apache.parquet.{Log => ApacheParquetLog} import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.hadoop._ import org.apache.parquet.hadoop.metadata.CompressionCodecName import org.apache.parquet.hadoop.util.ContextUtil import org.apache.parquet.schema.MessageType -import org.apache.parquet.{Log => ApacheParquetLog} import org.slf4j.bridge.SLF4JBridgeHandler +import org.apache.spark.{Logging, Partition => SparkPartition, SparkException} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{RDD, SqlNewHadoopPartition, SqlNewHadoopRDD} import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.util.LegacyTypeStringParser import org.apache.spark.sql.catalyst.{InternalRow, SqlParser, TableIdentifier} -import org.apache.spark.sql.execution.datasources.{PartitionSpec, _} +import org.apache.spark.sql.catalyst.util.LegacyTypeStringParser +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.{SerializableConfiguration, Utils} -import org.apache.spark.{Logging, Partition => SparkPartition, SparkException} private[sql] class DefaultSource extends BucketedHadoopFsRelationProvider with DataSourceRegister { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index d484403d1c641..1c773e69275db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql.{AnalysisException, SaveMode, SQLContext} import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.expressions.{RowOrdering, Alias, Attribute, Cast} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, RowOrdering} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala index cab6abde6da23..ae95b50e1ee76 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala @@ -21,9 +21,9 @@ import java.io.File import scala.collection.JavaConverters._ import scala.util.Try +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql.{SQLConf, SQLContext} import org.apache.spark.util.{Benchmark, Utils} -import org.apache.spark.{SparkConf, SparkContext} /** * Benchmark to measure parquet read performance. 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 5b13dbe47370e..d1b1c0d8d8bc2 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 @@ -24,11 +24,12 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, FunctionInfo} +import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.parse.EximUtil import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe + import org.apache.spark.Logging import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions._ @@ -38,7 +39,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.SparkQl import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper import org.apache.spark.sql.hive.client._ -import org.apache.spark.sql.hive.execution.{HiveNativeCommand, AnalyzeTable, DropTable, HiveScriptIOSchema} +import org.apache.spark.sql.hive.execution.{AnalyzeTable, DropTable, HiveNativeCommand, HiveScriptIOSchema} import org.apache.spark.sql.types._ import org.apache.spark.sql.AnalysisException diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala index 1c910051faccf..61e3f183bb42d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive import java.util.concurrent.atomic.AtomicLong +import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, NamedExpression, SortOrder} import org.apache.spark.sql.catalyst.optimizer.ProjectCollapsing @@ -26,7 +27,6 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor} import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation -import org.apache.spark.sql.{DataFrame, SQLContext} /** * A builder class used to convert a resolved logical plan into a SQL query string. Note that this 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 e76c18fa528f3..56cab1aee89df 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,21 +22,21 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Try import org.apache.hadoop.hive.ql.exec._ +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.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper -import org.apache.hadoop.hive.ql.udf.generic._ -import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions import org.apache.hadoop.hive.serde2.objectinspector.{ConstantObjectInspector, ObjectInspector, ObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions import org.apache.spark.Logging import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.{analysis, InternalRow} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.util.sequenceOption -import org.apache.spark.sql.catalyst.{InternalRow, analysis} import org.apache.spark.sql.hive.HiveShim._ import org.apache.spark.sql.hive.client.ClientWrapper import org.apache.spark.sql.types._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index d26cb48479066..033746d42f557 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -37,8 +37,8 @@ import org.apache.spark.sql.catalyst.expressions.ExpressionInfo import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.CacheTableCommand import org.apache.spark.sql.hive._ -import org.apache.spark.sql.hive.execution.HiveNativeCommand import org.apache.spark.sql.hive.client.ClientWrapper +import org.apache.spark.sql.hive.execution.HiveNativeCommand import org.apache.spark.util.{ShutdownHookManager, Utils} // SPARK-3729: Test key required to check for initialization errors with config. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index a2d283622ca52..e72a18a716b5c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -21,8 +21,8 @@ import scala.util.Try import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.catalyst.parser.ParseDriver import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.catalyst.parser.ParseDriver import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.hive.test.TestHiveSingleton diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala index 0e81acf532a03..9a8a9c51183da 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SQLTestUtils class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { import testImplicits._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.scala index cf4a3fdd88806..a5e209ac9db3b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.scala @@ -17,10 +17,10 @@ package org.apache.spark.sql.hive +import org.apache.spark.sql.{DataFrame, QueryTest} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.{DataFrame, QueryTest} abstract class SQLBuilderTest extends QueryTest with TestHiveSingleton { protected def checkSQL(e: Expression, expectedSQL: String): Unit = { 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 57358a07840e2..fd3339a66bec0 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 @@ -27,10 +27,10 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.datasources.DescribeCommand import org.apache.spark.sql.execution.{ExplainCommand, SetCommand} -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.execution.datasources.DescribeCommand import org.apache.spark.sql.hive.{InsertIntoHiveTable => LogicalInsertIntoHiveTable, SQLBuilder} +import org.apache.spark.sql.hive.test.TestHive /** * Allows the creations of tests that execute the same query against both hive diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala index 579da0291f291..7f1745705aaaf 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.sources import java.io.File -import org.apache.spark.sql.functions._ +import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.sql.{AnalysisException, QueryTest} class BucketedWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { import testImplicits._ 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 b186d297610e2..86f01d2168729 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -27,8 +27,8 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.Utils import org.apache.spark.streaming.scheduler.JobGenerator +import org.apache.spark.util.Utils private[streaming] class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala index 4e5baebaae04b..4ccc905b275d9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala @@ -25,7 +25,7 @@ import com.esotericsoftware.kryo.{Kryo, KryoSerializable} import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.spark.SparkConf -import org.apache.spark.serializer.{KryoOutputObjectOutputBridge, KryoInputObjectInputBridge} +import org.apache.spark.serializer.{KryoInputObjectInputBridge, KryoOutputObjectOutputBridge} import org.apache.spark.streaming.util.OpenHashMapBasedStateMap._ import org.apache.spark.util.collection.OpenHashMap diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala index ea32bbf95ce59..da0430e263b5f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala @@ -17,17 +17,16 @@ package org.apache.spark.streaming -import org.apache.spark.streaming.rdd.MapWithStateRDDRecord - import scala.collection.{immutable, mutable, Map} import scala.reflect.ClassTag import scala.util.Random import com.esotericsoftware.kryo.{Kryo, KryoSerializable} -import com.esotericsoftware.kryo.io.{Output, Input} +import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.serializer._ +import org.apache.spark.streaming.rdd.MapWithStateRDDRecord import org.apache.spark.streaming.util.{EmptyStateMap, OpenHashMapBasedStateMap, StateMap} class StateMapSuite extends SparkFunSuite { From 008a55828512056313da2626fd378e8aa1534790 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 10 Jan 2016 23:33:57 -0800 Subject: [PATCH 057/110] [SPARK-4628][BUILD] Add a resolver to MiMaBuild.scala for mqttv3(1.0.1). #10659 removed the repository `https://repo.eclipse.org/content/repositories/paho-releases` but it's needed by MiMa because `spark-streaming-mqtt(1.6.0)` depends on `mqttv3(1.0.1)` and it is provided by the removed repository and maven-central provide only `mqttv3(1.0.2)` for now. Otherwise, if `mqttv3(1.0.1)` is absent from the local repository, dev/mima should fail. JoshRosen Do you have any other better idea? Author: Kousuke Saruta Closes #10688 from sarutak/SPARK-4628-followup. --- project/MimaBuild.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 9ba9f8286f10c..41856443af49b 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -91,11 +91,16 @@ object MimaBuild { def mimaSettings(sparkHome: File, projectRef: ProjectRef) = { val organization = "org.apache.spark" + // The resolvers setting for MQTT Repository is needed for mqttv3(1.0.1) + // because spark-streaming-mqtt(1.6.0) depends on it. + // Remove the setting on updating previousSparkVersion. val previousSparkVersion = "1.6.0" val fullId = "spark-" + projectRef.project + "_2.10" mimaDefaultSettings ++ Seq(previousArtifact := Some(organization % fullId % previousSparkVersion), - binaryIssueFilters ++= ignoredABIProblems(sparkHome, version.value)) + binaryIssueFilters ++= ignoredABIProblems(sparkHome, version.value), + sbt.Keys.resolvers += + "MQTT Repository" at "https://repo.eclipse.org/content/repositories/paho-releases") } } From f13c7f8f7dc8766b0a42406b5c3639d6be55cf33 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 11 Jan 2016 00:31:29 -0800 Subject: [PATCH 058/110] [SPARK-12734][HOTFIX][TEST-MAVEN] Fix bug in Netty exclusions This is a hotfix for a build bug introduced by the Netty exclusion changes in #10672. We can't exclude `io.netty:netty` because Akka depends on it. There's not a direct conflict between `io.netty:netty` and `io.netty:netty-all`, because the former puts classes in the `org.jboss.netty` namespace while the latter uses the `io.netty` namespace. However, there still is a conflict between `org.jboss.netty:netty` and `io.netty:netty`, so we need to continue to exclude the JBoss version of that artifact. While the diff here looks somewhat large, note that this is only a revert of a some of the changes from #10672. You can see the net changes in pom.xml at https://github.com/apache/spark/compare/3119206b7188c23055621dfeaf6874f21c711a82...5211ab8#diff-600376dffeb79835ede4a0b285078036 Author: Josh Rosen Closes #10693 from JoshRosen/netty-hotfix. --- dev/deps/spark-deps-hadoop-2.2 | 1 + dev/deps/spark-deps-hadoop-2.3 | 1 + dev/deps/spark-deps-hadoop-2.4 | 1 + dev/deps/spark-deps-hadoop-2.6 | 1 + examples/pom.xml | 4 --- pom.xml | 50 +++++----------------------------- 6 files changed, 11 insertions(+), 47 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 13d1b0e950480..e4373f79f7922 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -142,6 +142,7 @@ metrics-graphite-3.1.2.jar metrics-json-3.1.2.jar metrics-jvm-3.1.2.jar minlog-1.2.jar +netty-3.8.0.Final.jar netty-all-4.0.29.Final.jar objenesis-1.2.jar opencsv-2.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index d7deaa0a24541..7478181406d07 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -133,6 +133,7 @@ metrics-json-3.1.2.jar metrics-jvm-3.1.2.jar minlog-1.2.jar mx4j-3.0.2.jar +netty-3.8.0.Final.jar netty-all-4.0.29.Final.jar objenesis-1.2.jar opencsv-2.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 7ad2212ed5ae7..faffb8bf398a5 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -134,6 +134,7 @@ metrics-json-3.1.2.jar metrics-jvm-3.1.2.jar minlog-1.2.jar mx4j-3.0.2.jar +netty-3.8.0.Final.jar netty-all-4.0.29.Final.jar objenesis-1.2.jar opencsv-2.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 7f8518927aec4..e703c7acd3876 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -140,6 +140,7 @@ metrics-json-3.1.2.jar metrics-jvm-3.1.2.jar minlog-1.2.jar mx4j-3.0.2.jar +netty-3.8.0.Final.jar netty-all-4.0.29.Final.jar objenesis-1.2.jar opencsv-2.3.jar diff --git a/examples/pom.xml b/examples/pom.xml index 6013085b10e84..1a0d5e5854642 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -111,10 +111,6 @@ org.jruby jruby-complete - - io.netty - netty - diff --git a/pom.xml b/pom.xml index cbed36c1eac16..06cccf1df0bb2 100644 --- a/pom.xml +++ b/pom.xml @@ -519,12 +519,6 @@ ${akka.group} akka-remote_${scala.binary.version} ${akka.version} - - - io.netty - netty - - ${akka.group} @@ -768,10 +762,6 @@ org.jboss.netty netty - - io.netty - netty - @@ -832,10 +822,6 @@ junit junit - - io.netty - netty - @@ -936,10 +922,6 @@ org.jboss.netty netty - - io.netty - netty - commons-logging commons-logging @@ -964,10 +946,6 @@ org.jboss.netty netty - - io.netty - netty - javax.servlet servlet-api @@ -997,10 +975,6 @@ org.jboss.netty netty - - io.netty - netty - javax.servlet servlet-api @@ -1029,10 +1003,6 @@ org.jboss.netty netty - - io.netty - netty - javax.servlet servlet-api @@ -1061,10 +1031,6 @@ org.jboss.netty netty - - io.netty - netty - javax.servlet servlet-api @@ -1085,10 +1051,6 @@ org.jboss.netty netty - - io.netty - netty - @@ -1832,13 +1794,15 @@ - io.netty:netty + org.jboss.netty - - - io.netty:netty:3.4.0.Final:*:test - true From f253feff62f3eb3cce22bbec0874f317a61b0092 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 11 Jan 2016 00:44:33 -0800 Subject: [PATCH 059/110] [SPARK-12539][FOLLOW-UP] always sort in partitioning writer address comments in #10498 , especially https://github.com/apache/spark/pull/10498#discussion_r49021259 Author: Wenchen Fan This patch had conflicts when merged, resolved by Committer: Reynold Xin Closes #10638 from cloud-fan/bucket-write. --- .../datasources/WriterContainer.scala | 192 +++++------------- .../apache/spark/sql/sources/interfaces.scala | 3 - 2 files changed, 48 insertions(+), 147 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index 40ecdb8e4403e..fff72872c13b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.UnsafeKVExternalSorter import org.apache.spark.sql.sources.{HadoopFsRelation, OutputWriter, OutputWriterFactory} -import org.apache.spark.sql.types.{IntegerType, StringType, StructType} +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.util.SerializableConfiguration @@ -349,67 +349,6 @@ private[sql] class DynamicPartitionWriterContainer( } } - private def sameBucket(key1: UnsafeRow, key2: UnsafeRow): Boolean = { - val bucketIdIndex = partitionColumns.length - if (key1.getInt(bucketIdIndex) != key2.getInt(bucketIdIndex)) { - false - } else { - var i = partitionColumns.length - 1 - while (i >= 0) { - val dt = partitionColumns(i).dataType - if (key1.get(i, dt) != key2.get(i, dt)) return false - i -= 1 - } - true - } - } - - private def sortBasedWrite( - sorter: UnsafeKVExternalSorter, - iterator: Iterator[InternalRow], - getSortingKey: UnsafeProjection, - getOutputRow: UnsafeProjection, - getPartitionString: UnsafeProjection, - outputWriters: java.util.HashMap[InternalRow, OutputWriter]): Unit = { - while (iterator.hasNext) { - val currentRow = iterator.next() - sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) - } - - logInfo(s"Sorting complete. Writing out partition files one at a time.") - - val needNewWriter: (UnsafeRow, UnsafeRow) => Boolean = if (sortColumns.isEmpty) { - (key1, key2) => key1 != key2 - } else { - (key1, key2) => key1 == null || !sameBucket(key1, key2) - } - - val sortedIterator = sorter.sortedIterator() - var currentKey: UnsafeRow = null - var currentWriter: OutputWriter = null - try { - while (sortedIterator.next()) { - if (needNewWriter(currentKey, sortedIterator.getKey)) { - if (currentWriter != null) { - currentWriter.close() - } - currentKey = sortedIterator.getKey.copy() - logDebug(s"Writing partition: $currentKey") - - // Either use an existing file from before, or open a new one. - currentWriter = outputWriters.remove(currentKey) - if (currentWriter == null) { - currentWriter = newOutputWriter(currentKey, getPartitionString) - } - } - - currentWriter.writeInternal(sortedIterator.getValue) - } - } finally { - if (currentWriter != null) { currentWriter.close() } - } - } - /** * Open and returns a new OutputWriter given a partition key and optional bucket id. * If bucket id is specified, we will append it to the end of the file name, but before the @@ -435,22 +374,18 @@ private[sql] class DynamicPartitionWriterContainer( } def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { - val outputWriters = new java.util.HashMap[InternalRow, OutputWriter] executorSideSetup(taskContext) - var outputWritersCleared = false - // We should first sort by partition columns, then bucket id, and finally sorting columns. - val getSortingKey = - UnsafeProjection.create(partitionColumns ++ bucketIdExpression ++ sortColumns, inputSchema) - - val sortingKeySchema = if (bucketSpec.isEmpty) { - StructType.fromAttributes(partitionColumns) - } else { // If it's bucketed, we should also consider bucket id as part of the key. - val fields = StructType.fromAttributes(partitionColumns) - .add("bucketId", IntegerType, nullable = false) ++ StructType.fromAttributes(sortColumns) - StructType(fields) - } + val sortingExpressions: Seq[Expression] = partitionColumns ++ bucketIdExpression ++ sortColumns + + val getSortingKey = UnsafeProjection.create(sortingExpressions, inputSchema) + + val sortingKeySchema = StructType(sortingExpressions.map { + case a: Attribute => StructField(a.name, a.dataType, a.nullable) + // The sorting expressions are all `Attribute` except bucket id. + case _ => StructField("bucketId", IntegerType, nullable = false) + }) // Returns the data columns to be written given an input row val getOutputRow = UnsafeProjection.create(dataColumns, inputSchema) @@ -461,54 +396,49 @@ private[sql] class DynamicPartitionWriterContainer( // If anything below fails, we should abort the task. try { - // If there is no sorting columns, we set sorter to null and try the hash-based writing first, - // and fill the sorter if there are too many writers and we need to fall back on sorting. - // If there are sorting columns, then we have to sort the data anyway, and no need to try the - // hash-based writing first. - var sorter: UnsafeKVExternalSorter = if (sortColumns.nonEmpty) { - new UnsafeKVExternalSorter( - sortingKeySchema, - StructType.fromAttributes(dataColumns), - SparkEnv.get.blockManager, - TaskContext.get().taskMemoryManager().pageSizeBytes) + // Sorts the data before write, so that we only need one writer at the same time. + // TODO: inject a local sort operator in planning. + val sorter = new UnsafeKVExternalSorter( + sortingKeySchema, + StructType.fromAttributes(dataColumns), + SparkEnv.get.blockManager, + TaskContext.get().taskMemoryManager().pageSizeBytes) + + while (iterator.hasNext) { + val currentRow = iterator.next() + sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) + } + + logInfo(s"Sorting complete. Writing out partition files one at a time.") + + val getBucketingKey: InternalRow => InternalRow = if (sortColumns.isEmpty) { + identity } else { - null + UnsafeProjection.create(sortingExpressions.dropRight(sortColumns.length).zipWithIndex.map { + case (expr, ordinal) => BoundReference(ordinal, expr.dataType, expr.nullable) + }) } - while (iterator.hasNext && sorter == null) { - val inputRow = iterator.next() - // When we reach here, the `sortColumns` must be empty, so the sorting key is hashing key. - val currentKey = getSortingKey(inputRow) - var currentWriter = outputWriters.get(currentKey) - - if (currentWriter == null) { - if (outputWriters.size < maxOpenFiles) { + + val sortedIterator = sorter.sortedIterator() + var currentKey: UnsafeRow = null + var currentWriter: OutputWriter = null + try { + while (sortedIterator.next()) { + val nextKey = getBucketingKey(sortedIterator.getKey).asInstanceOf[UnsafeRow] + if (currentKey != nextKey) { + if (currentWriter != null) { + currentWriter.close() + } + currentKey = nextKey.copy() + logDebug(s"Writing partition: $currentKey") + currentWriter = newOutputWriter(currentKey, getPartitionString) - outputWriters.put(currentKey.copy(), currentWriter) - currentWriter.writeInternal(getOutputRow(inputRow)) - } else { - logInfo(s"Maximum partitions reached, falling back on sorting.") - sorter = new UnsafeKVExternalSorter( - sortingKeySchema, - StructType.fromAttributes(dataColumns), - SparkEnv.get.blockManager, - TaskContext.get().taskMemoryManager().pageSizeBytes) - sorter.insertKV(currentKey, getOutputRow(inputRow)) } - } else { - currentWriter.writeInternal(getOutputRow(inputRow)) - } - } - // If the sorter is not null that means that we reached the maxFiles above and need to finish - // using external sort, or there are sorting columns and we need to sort the whole data set. - if (sorter != null) { - sortBasedWrite( - sorter, - iterator, - getSortingKey, - getOutputRow, - getPartitionString, - outputWriters) + currentWriter.writeInternal(sortedIterator.getValue) + } + } finally { + if (currentWriter != null) { currentWriter.close() } } commitTask() @@ -518,31 +448,5 @@ private[sql] class DynamicPartitionWriterContainer( abortTask() throw new SparkException("Task failed while writing rows.", cause) } - - def clearOutputWriters(): Unit = { - if (!outputWritersCleared) { - outputWriters.asScala.values.foreach(_.close()) - outputWriters.clear() - outputWritersCleared = true - } - } - - def commitTask(): Unit = { - try { - clearOutputWriters() - super.commitTask() - } catch { - case cause: Throwable => - throw new RuntimeException("Failed to commit task", cause) - } - } - - def abortTask(): Unit = { - try { - clearOutputWriters() - } finally { - super.abortTask() - } - } } } 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 index c35f33132f602..9f3607369c30f 100644 --- 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 @@ -162,7 +162,6 @@ trait HadoopFsRelationProvider { partitionColumns: Option[StructType], parameters: Map[String, String]): HadoopFsRelation - // TODO: expose bucket API to users. private[sql] def createRelation( sqlContext: SQLContext, paths: Array[String], @@ -370,7 +369,6 @@ abstract class OutputWriterFactory extends Serializable { dataSchema: StructType, context: TaskAttemptContext): OutputWriter - // TODO: expose bucket API to users. private[sql] def newInstance( path: String, bucketId: Option[Int], @@ -460,7 +458,6 @@ abstract class HadoopFsRelation private[sql]( private var _partitionSpec: PartitionSpec = _ - // TODO: expose bucket API to users. private[sql] def bucketSpec: Option[BucketSpec] = None private class FileStatusCache { From bd723bd53d9a28239b60939a248a4ea13340aad8 Mon Sep 17 00:00:00 2001 From: Udo Klein Date: Mon, 11 Jan 2016 09:30:08 +0000 Subject: [PATCH 060/110] removed lambda from sortByKey() According to the documentation the sortByKey method does not take a lambda as an argument, thus the example is flawed. Removed the argument completely as this will default to ascending sort. Author: Udo Klein Closes #10640 from udoklein/patch-1. --- examples/src/main/python/sort.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py index f6b0ecb02c100..b6c2916254056 100755 --- a/examples/src/main/python/sort.py +++ b/examples/src/main/python/sort.py @@ -30,7 +30,7 @@ lines = sc.textFile(sys.argv[1], 1) sortedCount = lines.flatMap(lambda x: x.split(' ')) \ .map(lambda x: (int(x), 1)) \ - .sortByKey(lambda x: x) + .sortByKey() # This is just a demo on how to bring all the sorted data back to a single node. # In reality, we wouldn't want to collect all the data to the driver node. output = sortedCount.collect() From 8fe928b4fe380ba527164bd413402abfed13c0e1 Mon Sep 17 00:00:00 2001 From: BrianLondon Date: Mon, 11 Jan 2016 09:32:06 +0000 Subject: [PATCH 061/110] [SPARK-12269][STREAMING][KINESIS] Update aws-java-sdk version The current Spark Streaming kinesis connector references a quite old version 1.9.40 of the AWS Java SDK (1.10.40 is current). Numerous AWS features including Kinesis Firehose are unavailable in 1.9. Those two versions of the AWS SDK in turn require conflicting versions of Jackson (2.4.4 and 2.5.3 respectively) such that one cannot include the current AWS SDK in a project that also uses the Spark Streaming Kinesis ASL. Author: BrianLondon Closes #10256 from BrianLondon/master. --- dev/deps/spark-deps-hadoop-2.2 | 8 ++++---- dev/deps/spark-deps-hadoop-2.3 | 8 ++++---- dev/deps/spark-deps-hadoop-2.4 | 8 ++++---- dev/deps/spark-deps-hadoop-2.6 | 8 ++++---- pom.xml | 6 +++--- 5 files changed, 19 insertions(+), 19 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index e4373f79f7922..cd3ff293502ae 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -84,13 +84,13 @@ hadoop-yarn-server-web-proxy-2.2.0.jar httpclient-4.3.2.jar httpcore-4.3.2.jar ivy-2.4.0.jar -jackson-annotations-2.4.4.jar -jackson-core-2.4.4.jar +jackson-annotations-2.5.3.jar +jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.4.4.jar +jackson-databind-2.5.3.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.10-2.4.4.jar +jackson-module-scala_2.10-2.5.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar jansi-1.4.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 7478181406d07..0985089ccea61 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -79,13 +79,13 @@ hadoop-yarn-server-web-proxy-2.3.0.jar httpclient-4.3.2.jar httpcore-4.3.2.jar ivy-2.4.0.jar -jackson-annotations-2.4.4.jar -jackson-core-2.4.4.jar +jackson-annotations-2.5.3.jar +jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.4.4.jar +jackson-databind-2.5.3.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.10-2.4.4.jar +jackson-module-scala_2.10-2.5.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar jansi-1.4.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index faffb8bf398a5..50f062601c02b 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -79,13 +79,13 @@ hadoop-yarn-server-web-proxy-2.4.0.jar httpclient-4.3.2.jar httpcore-4.3.2.jar ivy-2.4.0.jar -jackson-annotations-2.4.4.jar -jackson-core-2.4.4.jar +jackson-annotations-2.5.3.jar +jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.4.4.jar +jackson-databind-2.5.3.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.10-2.4.4.jar +jackson-module-scala_2.10-2.5.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar jansi-1.4.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index e703c7acd3876..2b6ca983ad65e 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -85,13 +85,13 @@ htrace-core-3.0.4.jar httpclient-4.3.2.jar httpcore-4.3.2.jar ivy-2.4.0.jar -jackson-annotations-2.4.4.jar -jackson-core-2.4.4.jar +jackson-annotations-2.5.3.jar +jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.4.4.jar +jackson-databind-2.5.3.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.10-2.4.4.jar +jackson-module-scala_2.10-2.5.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar jansi-1.4.jar diff --git a/pom.xml b/pom.xml index 06cccf1df0bb2..fc5cf970e0601 100644 --- a/pom.xml +++ b/pom.xml @@ -152,9 +152,9 @@ 1.7.7 hadoop2 0.7.1 - 1.4.0 + 1.6.1 - 0.10.1 + 0.10.2 4.3.2 @@ -167,7 +167,7 @@ ${scala.version} org.scala-lang 1.9.13 - 2.4.4 + 2.5.3 1.1.2 1.1.2 1.2.0-incubating From 9559ac5f74434cf4bf611bdcde9a216d39799826 Mon Sep 17 00:00:00 2001 From: Anatoliy Plastinin Date: Mon, 11 Jan 2016 10:28:57 -0800 Subject: [PATCH 062/110] [SPARK-12744][SQL] Change parsing JSON integers to timestamps to treat integers as number of seconds JIRA: https://issues.apache.org/jira/browse/SPARK-12744 This PR makes parsing JSON integers to timestamps consistent with casting behavior. Author: Anatoliy Plastinin Closes #10687 from antlypls/fix-json-timestamp-parsing. --- .../datasources/json/JacksonParser.scala | 2 +- .../execution/datasources/json/JsonSuite.scala | 17 +++++++++++++++-- .../datasources/json/TestJsonData.scala | 4 ++++ 3 files changed, 20 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala index 2e3fe3da15389..b2f5c1e96421d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala @@ -90,7 +90,7 @@ object JacksonParser { DateTimeUtils.stringToTime(parser.getText).getTime * 1000L case (VALUE_NUMBER_INT, TimestampType) => - parser.getLongValue * 1000L + parser.getLongValue * 1000000L case (_, StringType) => val writer = new ByteArrayOutputStream() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index b3b6b7df0c1d1..4ab148065a476 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -83,9 +83,9 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val doubleNumber: Double = 1.7976931348623157E308d checkTypePromotion(doubleNumber.toDouble, enforceCorrectType(doubleNumber, DoubleType)) - checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber)), + checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber * 1000L)), enforceCorrectType(intNumber, TimestampType)) - checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber.toLong)), + checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber.toLong * 1000L)), enforceCorrectType(intNumber.toLong, TimestampType)) val strTime = "2014-09-30 12:34:56" checkTypePromotion(DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf(strTime)), @@ -1465,4 +1465,17 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } } + test("Casting long as timestamp") { + withTempTable("jsonTable") { + val schema = (new StructType).add("ts", TimestampType) + val jsonDF = sqlContext.read.schema(schema).json(timestampAsLong) + + jsonDF.registerTempTable("jsonTable") + + checkAnswer( + sql("select ts from jsonTable"), + Row(java.sql.Timestamp.valueOf("2016-01-02 03:04:05")) + ) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala index cb61f7eeca0de..a0836058d3c74 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala @@ -205,6 +205,10 @@ private[json] trait TestJsonData { """{"b": [{"c": {}}]}""" :: """]""" :: Nil) + def timestampAsLong: RDD[String] = + sqlContext.sparkContext.parallelize( + """{"ts":1451732645}""" :: Nil) + lazy val singleRow: RDD[String] = sqlContext.sparkContext.parallelize("""{"a":123}""" :: Nil) def empty: RDD[String] = sqlContext.sparkContext.parallelize(Seq[String]()) From b313badaa049f847f33663c61cd70ee2f2cbebac Mon Sep 17 00:00:00 2001 From: Jacek Laskowski Date: Mon, 11 Jan 2016 11:29:15 -0800 Subject: [PATCH 063/110] [STREAMING][MINOR] Typo fixes Author: Jacek Laskowski Closes #10698 from jaceklaskowski/streaming-kafka-typo-fixes. --- .../scala/org/apache/spark/streaming/kafka/KafkaCluster.scala | 2 +- .../main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala index c4e18d92eefa9..d7885d7cc1ae1 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala @@ -385,7 +385,7 @@ object KafkaCluster { val seedBrokers: Array[(String, Int)] = brokers.split(",").map { hp => val hpa = hp.split(":") if (hpa.size == 1) { - throw new SparkException(s"Broker not the in correct format of : [$brokers]") + throw new SparkException(s"Broker not in the correct format of : [$brokers]") } (hpa(0), hpa(1).toInt) } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 603be22818206..4eb155645867b 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -156,7 +156,7 @@ class KafkaRDD[ var requestOffset = part.fromOffset var iter: Iterator[MessageAndOffset] = null - // The idea is to use the provided preferred host, except on task retry atttempts, + // The idea is to use the provided preferred host, except on task retry attempts, // to minimize number of kafka metadata requests private def connectLeader: SimpleConsumer = { if (context.attemptNumber > 0) { From a44991453a43615028083ba9546f5cd93112f6bd Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 11 Jan 2016 12:56:43 -0800 Subject: [PATCH 064/110] [SPARK-12734][HOTFIX] Build changes must trigger all tests; clean after install in dep tests This patch fixes a build/test issue caused by the combination of #10672 and a latent issue in the original `dev/test-dependencies` script. First, changes which _only_ touched build files were not triggering full Jenkins runs, making it possible for a build change to be merged even though it could cause failures in other tests. The `root` build module now depends on `build`, so all tests will now be run whenever a build-related file is changed. I also added a `clean` step to the Maven install step in `dev/test-dependencies` in order to address an issue where the dummy JARs stuck around and caused "multiple assembly JARs found" errors in tests. /cc zsxwing Author: Josh Rosen Closes #10704 from JoshRosen/fix-build-test-problems. --- dev/sparktestsupport/modules.py | 2 +- dev/test-dependencies.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 1fc6596164124..93a8c15e3ec30 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -426,7 +426,7 @@ def contains_file(self, filename): # No other modules should directly depend on this module. root = Module( name="root", - dependencies=[], + dependencies=[build], # Changes to build should trigger all tests. source_file_regexes=[], # In order to run all of the tests, enable every test profile: build_profile_flags=list(set( diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index def87aa4087e3..3cb5d2be2a91a 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -70,7 +70,7 @@ $MVN -q versions:set -DnewVersion=$TEMP_VERSION -DgenerateBackupPoms=false > /de # Generate manifests for each Hadoop profile: for HADOOP_PROFILE in "${HADOOP_PROFILES[@]}"; do echo "Performing Maven install for $HADOOP_PROFILE" - $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE jar:jar jar:test-jar install:install -q + $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE jar:jar jar:test-jar install:install clean -q echo "Performing Maven validate for $HADOOP_PROFILE" $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE validate -q From a767ee8a0599f5482717493a3298413c65d8ff89 Mon Sep 17 00:00:00 2001 From: Brandon Bradley Date: Mon, 11 Jan 2016 14:21:50 -0800 Subject: [PATCH 065/110] [SPARK-12758][SQL] add note to Spark SQL Migration guide about TimestampType casting Warning users about casting changes. Author: Brandon Bradley Closes #10708 from blbradley/spark-12758. --- docs/sql-programming-guide.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index b058833616433..bc89c781562bd 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -2151,6 +2151,11 @@ options. ... {% endhighlight %} + - From Spark 1.6, LongType casts to TimestampType expect seconds instead of microseconds. This + change was made to match the behavior of Hive 1.2 for more consistent type casting to TimestampType + from numeric types. See [SPARK-11724](https://issues.apache.org/jira/browse/SPARK-11724) for + details. + ## Upgrading From Spark SQL 1.4 to 1.5 - Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with From ee4ee02b86be8756a6d895a2e23e80862134a6d3 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Mon, 11 Jan 2016 14:43:25 -0800 Subject: [PATCH 066/110] [SPARK-12603][MLLIB] PySpark MLlib GaussianMixtureModel should support single instance predict/predictSoft PySpark MLlib ```GaussianMixtureModel``` should support single instance ```predict/predictSoft``` just like Scala do. Author: Yanbo Liang Closes #10552 from yanboliang/spark-12603. --- .../python/mllib/gaussian_mixture_model.py | 4 +++ .../examples/mllib/DenseGaussianMixture.scala | 6 ++++ .../python/GaussianMixtureModelWrapper.scala | 4 +++ .../clustering/GaussianMixtureModel.scala | 2 +- python/pyspark/mllib/clustering.py | 35 ++++++++++++------- 5 files changed, 37 insertions(+), 14 deletions(-) diff --git a/examples/src/main/python/mllib/gaussian_mixture_model.py b/examples/src/main/python/mllib/gaussian_mixture_model.py index 2cb8010cdc07f..69e836fc1d06a 100644 --- a/examples/src/main/python/mllib/gaussian_mixture_model.py +++ b/examples/src/main/python/mllib/gaussian_mixture_model.py @@ -62,5 +62,9 @@ def parseVector(line): for i in range(args.k): print(("weight = ", model.weights[i], "mu = ", model.gaussians[i].mu, "sigma = ", model.gaussians[i].sigma.toArray())) + print("\n") + print(("The membership value of each vector to all mixture components (first 100): ", + model.predictSoft(data).take(100))) + print("\n") print(("Cluster labels (first 100): ", model.predict(data).take(100))) sc.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala index 1fce4ba7efd60..90b817b23e156 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala @@ -58,6 +58,12 @@ object DenseGaussianMixture { (clusters.weights(i), clusters.gaussians(i).mu, clusters.gaussians(i).sigma)) } + println("The membership value of each vector to all mixture components (first <= 100):") + val membership = clusters.predictSoft(data) + membership.take(100).foreach { x => + print(" " + x.mkString(",")) + } + println() println("Cluster labels (first <= 100):") val clusterLabels = clusters.predict(data) clusterLabels.take(100).foreach { x => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/GaussianMixtureModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/GaussianMixtureModelWrapper.scala index 6a3b20c88d2d2..a689b09341450 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/GaussianMixtureModelWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/GaussianMixtureModelWrapper.scala @@ -40,5 +40,9 @@ private[python] class GaussianMixtureModelWrapper(model: GaussianMixtureModel) { SerDe.dumps(JavaConverters.seqAsJavaListConverter(modelGaussians).asJava) } + def predictSoft(point: Vector): Vector = { + Vectors.dense(model.predictSoft(point)) + } + def save(sc: SparkContext, path: String): Unit = model.save(sc, path) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index 16bc45bcb627f..42fe27024f8fc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -75,7 +75,7 @@ class GaussianMixtureModel @Since("1.3.0") ( */ @Since("1.5.0") def predict(point: Vector): Int = { - val r = computeSoftAssignments(point.toBreeze.toDenseVector, gaussians, weights, k) + val r = predictSoft(point) r.indexOf(r.max) } diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index d22a7f4c3b167..580cb512d8025 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -202,16 +202,25 @@ class GaussianMixtureModel(JavaModelWrapper, JavaSaveable, JavaLoader): >>> clusterdata_1 = sc.parallelize(array([-0.1,-0.05,-0.01,-0.1, ... 0.9,0.8,0.75,0.935, - ... -0.83,-0.68,-0.91,-0.76 ]).reshape(6, 2)) + ... -0.83,-0.68,-0.91,-0.76 ]).reshape(6, 2), 2) >>> model = GaussianMixture.train(clusterdata_1, 3, convergenceTol=0.0001, ... maxIterations=50, seed=10) >>> labels = model.predict(clusterdata_1).collect() >>> labels[0]==labels[1] False >>> labels[1]==labels[2] - True + False >>> labels[4]==labels[5] True + >>> model.predict([-0.1,-0.05]) + 0 + >>> softPredicted = model.predictSoft([-0.1,-0.05]) + >>> abs(softPredicted[0] - 1.0) < 0.001 + True + >>> abs(softPredicted[1] - 0.0) < 0.001 + True + >>> abs(softPredicted[2] - 0.0) < 0.001 + True >>> path = tempfile.mkdtemp() >>> model.save(sc, path) @@ -277,26 +286,27 @@ def k(self): @since('1.3.0') def predict(self, x): """ - Find the cluster to which the points in 'x' has maximum membership - in this model. + Find the cluster to which the point 'x' or each point in RDD 'x' + has maximum membership in this model. - :param x: RDD of data points. - :return: cluster_labels. RDD of cluster labels. + :param x: vector or RDD of vector represents data points. + :return: cluster label or RDD of cluster labels. """ if isinstance(x, RDD): cluster_labels = self.predictSoft(x).map(lambda z: z.index(max(z))) return cluster_labels else: - raise TypeError("x should be represented by an RDD, " - "but got %s." % type(x)) + z = self.predictSoft(x) + return z.argmax() @since('1.3.0') def predictSoft(self, x): """ - Find the membership of each point in 'x' to all mixture components. + Find the membership of point 'x' or each point in RDD 'x' to all mixture components. - :param x: RDD of data points. - :return: membership_matrix. RDD of array of double values. + :param x: vector or RDD of vector represents data points. + :return: the membership value to all mixture components for vector 'x' + or each vector in RDD 'x'. """ if isinstance(x, RDD): means, sigmas = zip(*[(g.mu, g.sigma) for g in self.gaussians]) @@ -304,8 +314,7 @@ def predictSoft(self, x): _convert_to_vector(self.weights), means, sigmas) return membership_matrix.map(lambda x: pyarray.array('d', x)) else: - raise TypeError("x should be represented by an RDD, " - "but got %s." % type(x)) + return self.call("predictSoft", _convert_to_vector(x)).toArray() @classmethod @since('1.5.0') From 4f8eefa36bb90812aac61ac7a762c9452de666bf Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Mon, 11 Jan 2016 14:48:35 -0800 Subject: [PATCH 067/110] [SPARK-12685][MLLIB] word2vec trainWordsCount gets overflow jira: https://issues.apache.org/jira/browse/SPARK-12685 the log of `word2vec` reports trainWordsCount = -785727483 during computation over a large dataset. Update the priority as it will affect the computation process. `alpha = learningRate * (1 - numPartitions * wordCount.toDouble / (trainWordsCount + 1))` Author: Yuhao Yang Closes #10627 from hhbyyh/w2voverflow. --- .../scala/org/apache/spark/mllib/feature/Word2Vec.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 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 a7e1b76df6a7d..dc5d070890d5d 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 @@ -151,7 +151,7 @@ class Word2Vec extends Serializable with Logging { /** context words from [-window, window] */ private var window = 5 - private var trainWordsCount = 0 + private var trainWordsCount = 0L private var vocabSize = 0 @transient private var vocab: Array[VocabWord] = null @transient private var vocabHash = mutable.HashMap.empty[String, Int] @@ -159,13 +159,13 @@ class Word2Vec extends Serializable with Logging { private def learnVocab(words: RDD[String]): Unit = { vocab = words.map(w => (w, 1)) .reduceByKey(_ + _) + .filter(_._2 >= minCount) .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) @@ -179,7 +179,7 @@ class Word2Vec extends Serializable with Logging { trainWordsCount += vocab(a).cn a += 1 } - logInfo("trainWordsCount = " + trainWordsCount) + logInfo(s"vocabSize = $vocabSize, trainWordsCount = $trainWordsCount") } private def createExpTable(): Array[Float] = { @@ -332,7 +332,7 @@ class Word2Vec extends Serializable with Logging { 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((bcSyn0Global.value, bcSyn1Global.value, 0, 0)) { + val model = iter.foldLeft((bcSyn0Global.value, bcSyn1Global.value, 0L, 0L)) { case ((syn0, syn1, lastWordCount, wordCount), sentence) => var lwc = lastWordCount var wc = wordCount From bbea88852ce6a3127d071ca40dbca2d042f9fbcf Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Mon, 11 Jan 2016 14:55:44 -0800 Subject: [PATCH 068/110] [SPARK-10809][MLLIB] Single-document topicDistributions method for LocalLDAModel jira: https://issues.apache.org/jira/browse/SPARK-10809 We could provide a single-document topicDistributions method for LocalLDAModel to allow for quick queries which avoid RDD operations. Currently, the user must use an RDD of documents. add some missing assert too. Author: Yuhao Yang Closes #9484 from hhbyyh/ldaTopicPre. --- .../spark/mllib/clustering/LDAModel.scala | 26 +++++++++++++++++++ .../spark/mllib/clustering/LDASuite.scala | 15 ++++++++--- 2 files changed, 38 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 2fce3ff641101..b30ecb80209d9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -387,6 +387,32 @@ class LocalLDAModel private[spark] ( } } + /** + * Predicts the topic mixture distribution for a document (often called "theta" in the + * literature). Returns a vector of zeros for an empty document. + * + * Note this means to allow quick query for single document. For batch documents, please refer + * to [[topicDistributions()]] to avoid overhead. + * + * @param document document to predict topic mixture distributions for + * @return topic mixture distribution for the document + */ + @Since("2.0.0") + def topicDistribution(document: Vector): Vector = { + val expElogbeta = exp(LDAUtils.dirichletExpectation(topicsMatrix.toBreeze.toDenseMatrix.t).t) + if (document.numNonzeros == 0) { + Vectors.zeros(this.k) + } else { + val (gamma, _) = OnlineLDAOptimizer.variationalTopicInference( + document, + expElogbeta, + this.docConcentration.toBreeze, + gammaShape, + this.k) + Vectors.dense(normalize(gamma, 1.0).toArray) + } + } + /** * Java-friendly version of [[topicDistributions]] */ diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index faef60e084cc1..ea23196d2c801 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -366,7 +366,8 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext { (0, 0.99504), (1, 0.99504), (1, 0.99504), (1, 0.99504)) - val actualPredictions = ldaModel.topicDistributions(docs).map { case (id, topics) => + val actualPredictions = ldaModel.topicDistributions(docs).cache() + val topTopics = actualPredictions.map { case (id, topics) => // convert results to expectedPredictions format, which only has highest probability topic val topicsBz = topics.toBreeze.toDenseVector (id, (argmax(topicsBz), max(topicsBz))) @@ -374,9 +375,17 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext { .values .collect() - expectedPredictions.zip(actualPredictions).forall { case (expected, actual) => - expected._1 === actual._1 && (expected._2 ~== actual._2 relTol 1E-3D) + expectedPredictions.zip(topTopics).foreach { case (expected, actual) => + assert(expected._1 === actual._1 && (expected._2 ~== actual._2 relTol 1E-3D)) } + + docs.collect() + .map(doc => ldaModel.topicDistribution(doc._2)) + .zip(actualPredictions.map(_._2).collect()) + .foreach { case (single, batch) => + assert(single ~== batch relTol 1E-3D) + } + actualPredictions.unpersist() } test("OnlineLDAOptimizer with asymmetric prior") { From fe9eb0b0ce397aeb40a32f8231d2ce8c17d7a609 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 11 Jan 2016 16:29:37 -0800 Subject: [PATCH 069/110] [SPARK-12576][SQL] Enable expression parsing in CatalystQl The PR allows us to use the new SQL parser to parse SQL expressions such as: ```1 + sin(x*x)``` We enable this functionality in this PR, but we will not start using this actively yet. This will be done as soon as we have reached grammar parity with the existing parser stack. cc rxin Author: Herman van Hovell Closes #10649 from hvanhovell/SPARK-12576. --- .../sql/catalyst/parser/SelectClauseParser.g | 7 + .../spark/sql/catalyst/CatalystQl.scala | 59 ++++--- .../sql/catalyst/parser/ParseDriver.scala | 24 ++- .../spark/sql/catalyst/CatalystQlSuite.scala | 151 ++++++++++++++++-- .../spark/sql/hive/ExtendedHiveQlParser.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 4 +- .../org/apache/spark/sql/hive/HiveQl.scala | 19 +-- .../spark/sql/hive/ErrorPositionSuite.scala | 5 +- .../apache/spark/sql/hive/HiveQlSuite.scala | 2 +- 9 files changed, 217 insertions(+), 56 deletions(-) diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g index 2d2bafb1ee34f..f18b6ec496f8f 100644 --- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g @@ -131,6 +131,13 @@ selectItem : (tableAllColumns) => tableAllColumns -> ^(TOK_SELEXPR tableAllColumns) | + namedExpression + ; + +namedExpression +@init { gParent.pushMsg("select named expression", state); } +@after { gParent.popMsg(state); } + : ( expression ((KW_AS? identifier) | (KW_AS LPAREN identifier (COMMA identifier)* RPAREN))? ) -> ^(TOK_SELEXPR expression identifier*) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala index 2e3cc0bfde7c7..c87b6c8e95436 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala @@ -30,6 +30,12 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.util.random.RandomSampler +private[sql] object CatalystQl { + val parser = new CatalystQl + def parseExpression(sql: String): Expression = parser.parseExpression(sql) + def parseTableIdentifier(sql: String): TableIdentifier = parser.parseTableIdentifier(sql) +} + /** * This class translates a HQL String to a Catalyst [[LogicalPlan]] or [[Expression]]. */ @@ -41,16 +47,13 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) { } } - /** - * Returns the AST for the given SQL string. + * The safeParse method allows a user to focus on the parsing/AST transformation logic. This + * method will take care of possible errors during the parsing process. */ - protected def getAst(sql: String): ASTNode = ParseDriver.parse(sql, conf) - - /** Creates LogicalPlan for a given HiveQL string. */ - def createPlan(sql: String): LogicalPlan = { + protected def safeParse[T](sql: String, ast: ASTNode)(toResult: ASTNode => T): T = { try { - createPlan(sql, ParseDriver.parse(sql, conf)) + toResult(ast) } catch { case e: MatchError => throw e case e: AnalysisException => throw e @@ -58,26 +61,39 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) { throw new AnalysisException(e.getMessage) case e: NotImplementedError => throw new AnalysisException( - s""" - |Unsupported language features in query: $sql - |${getAst(sql).treeString} + s"""Unsupported language features in query + |== SQL == + |$sql + |== AST == + |${ast.treeString} + |== Error == |$e + |== Stacktrace == |${e.getStackTrace.head} """.stripMargin) } } - protected def createPlan(sql: String, tree: ASTNode): LogicalPlan = nodeToPlan(tree) - - def parseDdl(ddl: String): Seq[Attribute] = { - val tree = getAst(ddl) - assert(tree.text == "TOK_CREATETABLE", "Only CREATE TABLE supported.") - val tableOps = tree.children - val colList = tableOps - .find(_.text == "TOK_TABCOLLIST") - .getOrElse(sys.error("No columnList!")) - - colList.children.map(nodeToAttribute) + /** Creates LogicalPlan for a given SQL string. */ + def parsePlan(sql: String): LogicalPlan = + safeParse(sql, ParseDriver.parsePlan(sql, conf))(nodeToPlan) + + /** Creates Expression for a given SQL string. */ + def parseExpression(sql: String): Expression = + safeParse(sql, ParseDriver.parseExpression(sql, conf))(selExprNodeToExpr(_).get) + + /** Creates TableIdentifier for a given SQL string. */ + def parseTableIdentifier(sql: String): TableIdentifier = + safeParse(sql, ParseDriver.parseTableName(sql, conf))(extractTableIdent) + + def parseDdl(sql: String): Seq[Attribute] = { + safeParse(sql, ParseDriver.parseExpression(sql, conf)) { ast => + val Token("TOK_CREATETABLE", children) = ast + children + .find(_.text == "TOK_TABCOLLIST") + .getOrElse(sys.error("No columnList!")) + .flatMap(_.children.map(nodeToAttribute)) + } } protected def getClauses( @@ -187,7 +203,6 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val keyMap = keyASTs.zipWithIndex.toMap val bitmasks: Seq[Int] = setASTs.map { - case Token("TOK_GROUPING_SETS_EXPRESSION", null) => 0 case Token("TOK_GROUPING_SETS_EXPRESSION", columns) => columns.foldLeft(0)((bitmap, col) => { val keyIndex = keyMap.find(_._1.treeEquals(col)).map(_._2) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index 0e93af8b92cd2..f8e4f21451192 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -28,7 +28,25 @@ import org.apache.spark.sql.AnalysisException * This is based on Hive's org.apache.hadoop.hive.ql.parse.ParseDriver */ object ParseDriver extends Logging { - def parse(command: String, conf: ParserConf): ASTNode = { + /** Create an LogicalPlan ASTNode from a SQL command. */ + def parsePlan(command: String, conf: ParserConf): ASTNode = parse(command, conf) { parser => + parser.statement().getTree + } + + /** Create an Expression ASTNode from a SQL command. */ + def parseExpression(command: String, conf: ParserConf): ASTNode = parse(command, conf) { parser => + parser.namedExpression().getTree + } + + /** Create an TableIdentifier ASTNode from a SQL command. */ + def parseTableName(command: String, conf: ParserConf): ASTNode = parse(command, conf) { parser => + parser.tableName().getTree + } + + private def parse( + command: String, + conf: ParserConf)( + toTree: SparkSqlParser => CommonTree): ASTNode = { logInfo(s"Parsing command: $command") // Setup error collection. @@ -44,7 +62,7 @@ object ParseDriver extends Logging { parser.configure(conf, reporter) try { - val result = parser.statement() + val result = toTree(parser) // Check errors. reporter.checkForErrors() @@ -57,7 +75,7 @@ object ParseDriver extends Logging { if (tree.token != null || tree.getChildCount == 0) tree else nonNullToken(tree.getChild(0).asInstanceOf[CommonTree]) } - val tree = nonNullToken(result.getTree) + val tree = nonNullToken(result) // Make sure all boundaries are set. tree.setUnknownTokenBoundaries() diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala index d7204c3488313..ba9d2524a9551 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala @@ -17,36 +17,157 @@ package org.apache.spark.sql.catalyst +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Project} +import org.apache.spark.unsafe.types.CalendarInterval class CatalystQlSuite extends PlanTest { val parser = new CatalystQl() + test("test case insensitive") { + val result = Project(UnresolvedAlias(Literal(1)):: Nil, OneRowRelation) + assert(result === parser.parsePlan("seLect 1")) + assert(result === parser.parsePlan("select 1")) + assert(result === parser.parsePlan("SELECT 1")) + } + + test("test NOT operator with comparison operations") { + val parsed = parser.parsePlan("SELECT NOT TRUE > TRUE") + val expected = Project( + UnresolvedAlias( + Not( + GreaterThan(Literal(true), Literal(true))) + ) :: Nil, + OneRowRelation) + comparePlans(parsed, expected) + } + + test("support hive interval literal") { + def checkInterval(sql: String, result: CalendarInterval): Unit = { + val parsed = parser.parsePlan(sql) + val expected = Project( + UnresolvedAlias( + Literal(result) + ) :: Nil, + OneRowRelation) + comparePlans(parsed, expected) + } + + def checkYearMonth(lit: String): Unit = { + checkInterval( + s"SELECT INTERVAL '$lit' YEAR TO MONTH", + CalendarInterval.fromYearMonthString(lit)) + } + + def checkDayTime(lit: String): Unit = { + checkInterval( + s"SELECT INTERVAL '$lit' DAY TO SECOND", + CalendarInterval.fromDayTimeString(lit)) + } + + def checkSingleUnit(lit: String, unit: String): Unit = { + checkInterval( + s"SELECT INTERVAL '$lit' $unit", + CalendarInterval.fromSingleUnitString(unit, lit)) + } + + checkYearMonth("123-10") + checkYearMonth("496-0") + checkYearMonth("-2-3") + checkYearMonth("-123-0") + + checkDayTime("99 11:22:33.123456789") + checkDayTime("-99 11:22:33.123456789") + checkDayTime("10 9:8:7.123456789") + checkDayTime("1 0:0:0") + checkDayTime("-1 0:0:0") + checkDayTime("1 0:0:1") + + for (unit <- Seq("year", "month", "day", "hour", "minute", "second")) { + checkSingleUnit("7", unit) + checkSingleUnit("-7", unit) + checkSingleUnit("0", unit) + } + + checkSingleUnit("13.123456789", "second") + checkSingleUnit("-13.123456789", "second") + } + + test("support scientific notation") { + def assertRight(input: String, output: Double): Unit = { + val parsed = parser.parsePlan("SELECT " + input) + val expected = Project( + UnresolvedAlias( + Literal(output) + ) :: Nil, + OneRowRelation) + comparePlans(parsed, expected) + } + + assertRight("9.0e1", 90) + assertRight("0.9e+2", 90) + assertRight("900e-1", 90) + assertRight("900.0E-1", 90) + assertRight("9.e+1", 90) + + intercept[AnalysisException](parser.parsePlan("SELECT .e3")) + } + + test("parse expressions") { + compareExpressions( + parser.parseExpression("prinln('hello', 'world')"), + UnresolvedFunction( + "prinln", Literal("hello") :: Literal("world") :: Nil, false)) + + compareExpressions( + parser.parseExpression("1 + r.r As q"), + Alias(Add(Literal(1), UnresolvedAttribute("r.r")), "q")()) + + compareExpressions( + parser.parseExpression("1 - f('o', o(bar))"), + Subtract(Literal(1), + UnresolvedFunction("f", + Literal("o") :: + UnresolvedFunction("o", UnresolvedAttribute("bar") :: Nil, false) :: + Nil, false))) + } + + test("table identifier") { + assert(TableIdentifier("q") === parser.parseTableIdentifier("q")) + assert(TableIdentifier("q", Some("d")) === parser.parseTableIdentifier("d.q")) + intercept[AnalysisException](parser.parseTableIdentifier("")) + // TODO parser swallows third identifier. + // intercept[AnalysisException](parser.parseTableIdentifier("d.q.g")) + } + test("parse union/except/intersect") { - parser.createPlan("select * from t1 union all select * from t2") - parser.createPlan("select * from t1 union distinct select * from t2") - parser.createPlan("select * from t1 union select * from t2") - parser.createPlan("select * from t1 except select * from t2") - parser.createPlan("select * from t1 intersect select * from t2") - parser.createPlan("(select * from t1) union all (select * from t2)") - parser.createPlan("(select * from t1) union distinct (select * from t2)") - parser.createPlan("(select * from t1) union (select * from t2)") - parser.createPlan("select * from ((select * from t1) union (select * from t2)) t") + parser.parsePlan("select * from t1 union all select * from t2") + parser.parsePlan("select * from t1 union distinct select * from t2") + parser.parsePlan("select * from t1 union select * from t2") + parser.parsePlan("select * from t1 except select * from t2") + parser.parsePlan("select * from t1 intersect select * from t2") + parser.parsePlan("(select * from t1) union all (select * from t2)") + parser.parsePlan("(select * from t1) union distinct (select * from t2)") + parser.parsePlan("(select * from t1) union (select * from t2)") + parser.parsePlan("select * from ((select * from t1) union (select * from t2)) t") } test("window function: better support of parentheses") { - parser.createPlan("select sum(product + 1) over (partition by ((1) + (product / 2)) " + + parser.parsePlan("select sum(product + 1) over (partition by ((1) + (product / 2)) " + "order by 2) from windowData") - parser.createPlan("select sum(product + 1) over (partition by (1 + (product / 2)) " + + parser.parsePlan("select sum(product + 1) over (partition by (1 + (product / 2)) " + "order by 2) from windowData") - parser.createPlan("select sum(product + 1) over (partition by ((product / 2) + 1) " + + parser.parsePlan("select sum(product + 1) over (partition by ((product / 2) + 1) " + "order by 2) from windowData") - parser.createPlan("select sum(product + 1) over (partition by ((product) + (1)) order by 2) " + + parser.parsePlan("select sum(product + 1) over (partition by ((product) + (1)) order by 2) " + "from windowData") - parser.createPlan("select sum(product + 1) over (partition by ((product) + 1) order by 2) " + + parser.parsePlan("select sum(product + 1) over (partition by ((product) + 1) order by 2) " + "from windowData") - parser.createPlan("select sum(product + 1) over (partition by (product + (1)) order by 2) " + + parser.parsePlan("select sum(product + 1) over (partition by (product + (1)) order by 2) " + "from windowData") } } 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 395c8bff53f47..b22f424981325 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 @@ -38,7 +38,7 @@ private[hive] class ExtendedHiveQlParser extends AbstractSparkSQLParser { protected lazy val hiveQl: Parser[LogicalPlan] = restInput ^^ { - case statement => HiveQl.createPlan(statement.trim) + case statement => HiveQl.parsePlan(statement.trim) } protected lazy val dfs: Parser[LogicalPlan] = 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 43d84d507b20e..67228f3f3c9c9 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 @@ -414,8 +414,8 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive alias match { // because hive use things like `_c0` to build the expanded text // currently we cannot support view from "create view v1(c1) as ..." - case None => Subquery(table.name, HiveQl.createPlan(viewText)) - case Some(aliasText) => Subquery(aliasText, HiveQl.createPlan(viewText)) + case None => Subquery(table.name, HiveQl.parsePlan(viewText)) + case Some(aliasText) => Subquery(aliasText, HiveQl.parsePlan(viewText)) } } else { MetastoreRelation(qualifiedTableName.database, qualifiedTableName.name, alias)(table)(hive) 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 d1b1c0d8d8bc2..ca9ddf94c11a7 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 @@ -230,15 +230,16 @@ private[hive] object HiveQl extends SparkQl with Logging { CreateViewAsSelect(tableDesc, nodeToPlan(query), allowExist, replace, sql) } - protected override def createPlan( - sql: String, - node: ASTNode): LogicalPlan = { - if (nativeCommands.contains(node.text)) { - HiveNativeCommand(sql) - } else { - nodeToPlan(node) match { - case NativePlaceholder => HiveNativeCommand(sql) - case plan => plan + /** Creates LogicalPlan for a given SQL string. */ + override def parsePlan(sql: String): LogicalPlan = { + safeParse(sql, ParseDriver.parsePlan(sql, conf)) { ast => + if (nativeCommands.contains(ast.text)) { + HiveNativeCommand(sql) + } else { + nodeToPlan(ast) match { + case NativePlaceholder => HiveNativeCommand(sql) + case plan => plan + } } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index e72a18a716b5c..14a466cfe9486 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -117,9 +117,8 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd * @param token a unique token in the string that should be indicated by the exception */ def positionTest(name: String, query: String, token: String): Unit = { - def ast = ParseDriver.parse(query, hiveContext.conf) - def parseTree = - Try(quietly(ast.treeString)).getOrElse("") + def ast = ParseDriver.parsePlan(query, hiveContext.conf) + def parseTree = Try(quietly(ast.treeString)).getOrElse("") test(name) { val error = intercept[AnalysisException] { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala index f4a1a17422483..53d15c14cb3d5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.hive.client.{ExternalTable, HiveColumn, HiveTable, M class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { private def extractTableDesc(sql: String): (HiveTable, Boolean) = { - HiveQl.createPlan(sql).collect { + HiveQl.parsePlan(sql).collect { case CreateTableAsSelect(desc, child, allowExisting) => (desc, allowExisting) }.head } From 473907adf6e37855ee31d0703b43d7170e26b4b9 Mon Sep 17 00:00:00 2001 From: wangfei Date: Mon, 11 Jan 2016 18:18:44 -0800 Subject: [PATCH 070/110] [SPARK-12742][SQL] org.apache.spark.sql.hive.LogicalPlanToSQLSuite failure due to Table already exists exception ``` [info] Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.LogicalPlanToSQLSuite *** ABORTED *** (325 milliseconds) [info] org.apache.spark.sql.AnalysisException: Table `t1` already exists.; [info] at org.apache.spark.sql.DataFrameWriter.saveAsTable(DataFrameWriter.scala:296) [info] at org.apache.spark.sql.DataFrameWriter.saveAsTable(DataFrameWriter.scala:285) [info] at org.apache.spark.sql.hive.LogicalPlanToSQLSuite.beforeAll(LogicalPlanToSQLSuite.scala:33) [info] at org.scalatest.BeforeAndAfterAll$class.beforeAll(BeforeAndAfterAll.scala:187) [info] at org.apache.spark.sql.hive.LogicalPlanToSQLSuite.beforeAll(LogicalPlanToSQLSuite.scala:23) [info] at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:253) [info] at org.apache.spark.sql.hive.LogicalPlanToSQLSuite.run(LogicalPlanToSQLSuite.scala:23) [info] at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462) [info] at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:296) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:286) [info] at java.util.concurrent.FutureTask.run(FutureTask.java:266) [info] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [info] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [info] at java.lang.Thread.run(Thread.java:745) ``` /cc liancheng Author: wangfei Closes #10682 from scwf/fix-test. --- .../org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala index 9a8a9c51183da..2ee8150fb80d5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala @@ -24,6 +24,9 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { import testImplicits._ protected override def beforeAll(): Unit = { + sql("DROP TABLE IF EXISTS t0") + sql("DROP TABLE IF EXISTS t1") + sql("DROP TABLE IF EXISTS t2") sqlContext.range(10).write.saveAsTable("t0") sqlContext From 36d493509d32d14b54af62f5f65e8fa750e7413d Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 11 Jan 2016 18:42:26 -0800 Subject: [PATCH 071/110] [SPARK-12498][SQL][MINOR] BooleanSimplication simplification Scala syntax allows binary case classes to be used as infix operator in pattern matching. This PR makes use of this syntax sugar to make `BooleanSimplification` more readable. Author: Cheng Lian Closes #10445 from liancheng/boolean-simplification-simplification. --- .../sql/catalyst/expressions/literals.scala | 4 + .../sql/catalyst/optimizer/Optimizer.scala | 190 ++++++++---------- 2 files changed, 92 insertions(+), 102 deletions(-) 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 17351ef0685a9..e0b020330278b 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 @@ -28,6 +28,10 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types._ object Literal { + val TrueLiteral: Literal = Literal(true, BooleanType) + + val FalseLiteral: Literal = Literal(false, BooleanType) + def apply(v: Any): Literal = v match { case i: Int => Literal(i, IntegerType) case l: Long => Literal(l, LongType) 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 f8121a733a8d2..b70bc184d0a5e 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 @@ -21,6 +21,7 @@ import scala.collection.immutable.HashSet import org.apache.spark.sql.catalyst.analysis.{CleanupAliases, EliminateSubQueries} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.planning.ExtractFiltersAndInnerJoins import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, LeftOuter, LeftSemi, RightOuter} @@ -519,112 +520,97 @@ object OptimizeIn extends Rule[LogicalPlan] { object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsUp { - case and @ And(left, right) => (left, right) match { - // true && r => r - case (Literal(true, BooleanType), r) => r - // l && true => l - case (l, Literal(true, BooleanType)) => l - // false && r => false - case (Literal(false, BooleanType), _) => Literal(false) - // l && false => false - case (_, Literal(false, BooleanType)) => Literal(false) - // a && a => a - case (l, r) if l fastEquals r => l - // a && (not(a) || b) => a && b - case (l, Or(l1, r)) if (Not(l) == l1) => And(l, r) - case (l, Or(r, l1)) if (Not(l) == l1) => And(l, r) - case (Or(l, l1), r) if (l1 == Not(r)) => And(l, r) - case (Or(l1, l), r) if (l1 == Not(r)) => And(l, r) - // (a || b) && (a || c) => a || (b && c) - case _ => - // 1. Split left and right to get the disjunctive predicates, - // i.e. lhs = (a, b), rhs = (a, c) - // 2. Find the common predict between lhsSet and rhsSet, i.e. common = (a) - // 3. Remove common predict from lhsSet and rhsSet, i.e. ldiff = (b), rdiff = (c) - // 4. Apply the formula, get the optimized predicate: common || (ldiff && rdiff) - val lhs = splitDisjunctivePredicates(left) - val rhs = splitDisjunctivePredicates(right) - val common = lhs.filter(e => rhs.exists(e.semanticEquals(_))) - if (common.isEmpty) { - // No common factors, return the original predicate - and + case TrueLiteral And e => e + case e And TrueLiteral => e + case FalseLiteral Or e => e + case e Or FalseLiteral => e + + case FalseLiteral And _ => FalseLiteral + case _ And FalseLiteral => FalseLiteral + case TrueLiteral Or _ => TrueLiteral + case _ Or TrueLiteral => TrueLiteral + + case a And b if a.semanticEquals(b) => a + case a Or b if a.semanticEquals(b) => a + + case a And (b Or c) if Not(a).semanticEquals(b) => And(a, c) + case a And (b Or c) if Not(a).semanticEquals(c) => And(a, b) + case (a Or b) And c if a.semanticEquals(Not(c)) => And(b, c) + case (a Or b) And c if b.semanticEquals(Not(c)) => And(a, c) + + case a Or (b And c) if Not(a).semanticEquals(b) => Or(a, c) + case a Or (b And c) if Not(a).semanticEquals(c) => Or(a, b) + case (a And b) Or c if a.semanticEquals(Not(c)) => Or(b, c) + case (a And b) Or c if b.semanticEquals(Not(c)) => Or(a, c) + + // Common factor elimination for conjunction + case and @ (left And right) => + // 1. Split left and right to get the disjunctive predicates, + // i.e. lhs = (a, b), rhs = (a, c) + // 2. Find the common predict between lhsSet and rhsSet, i.e. common = (a) + // 3. Remove common predict from lhsSet and rhsSet, i.e. ldiff = (b), rdiff = (c) + // 4. Apply the formula, get the optimized predicate: common || (ldiff && rdiff) + val lhs = splitDisjunctivePredicates(left) + val rhs = splitDisjunctivePredicates(right) + val common = lhs.filter(e => rhs.exists(e.semanticEquals)) + if (common.isEmpty) { + // No common factors, return the original predicate + and + } else { + val ldiff = lhs.filterNot(e => common.exists(e.semanticEquals)) + val rdiff = rhs.filterNot(e => common.exists(e.semanticEquals)) + if (ldiff.isEmpty || rdiff.isEmpty) { + // (a || b || c || ...) && (a || b) => (a || b) + common.reduce(Or) } else { - val ldiff = lhs.filterNot(e => common.exists(e.semanticEquals(_))) - val rdiff = rhs.filterNot(e => common.exists(e.semanticEquals(_))) - if (ldiff.isEmpty || rdiff.isEmpty) { - // (a || b || c || ...) && (a || b) => (a || b) - common.reduce(Or) - } else { - // (a || b || c || ...) && (a || b || d || ...) => - // ((c || ...) && (d || ...)) || a || b - (common :+ And(ldiff.reduce(Or), rdiff.reduce(Or))).reduce(Or) - } + // (a || b || c || ...) && (a || b || d || ...) => + // ((c || ...) && (d || ...)) || a || b + (common :+ And(ldiff.reduce(Or), rdiff.reduce(Or))).reduce(Or) } - } // end of And(left, right) - - case or @ Or(left, right) => (left, right) match { - // true || r => true - case (Literal(true, BooleanType), _) => Literal(true) - // r || true => true - case (_, Literal(true, BooleanType)) => Literal(true) - // false || r => r - case (Literal(false, BooleanType), r) => r - // l || false => l - case (l, Literal(false, BooleanType)) => l - // a || a => a - case (l, r) if l fastEquals r => l - // (a && b) || (a && c) => a && (b || c) - case _ => - // 1. Split left and right to get the conjunctive predicates, - // i.e. lhs = (a, b), rhs = (a, c) - // 2. Find the common predict between lhsSet and rhsSet, i.e. common = (a) - // 3. Remove common predict from lhsSet and rhsSet, i.e. ldiff = (b), rdiff = (c) - // 4. Apply the formula, get the optimized predicate: common && (ldiff || rdiff) - val lhs = splitConjunctivePredicates(left) - val rhs = splitConjunctivePredicates(right) - val common = lhs.filter(e => rhs.exists(e.semanticEquals(_))) - if (common.isEmpty) { - // No common factors, return the original predicate - or + } + + // Common factor elimination for disjunction + case or @ (left Or right) => + // 1. Split left and right to get the conjunctive predicates, + // i.e. lhs = (a, b), rhs = (a, c) + // 2. Find the common predict between lhsSet and rhsSet, i.e. common = (a) + // 3. Remove common predict from lhsSet and rhsSet, i.e. ldiff = (b), rdiff = (c) + // 4. Apply the formula, get the optimized predicate: common && (ldiff || rdiff) + val lhs = splitConjunctivePredicates(left) + val rhs = splitConjunctivePredicates(right) + val common = lhs.filter(e => rhs.exists(e.semanticEquals)) + if (common.isEmpty) { + // No common factors, return the original predicate + or + } else { + val ldiff = lhs.filterNot(e => common.exists(e.semanticEquals)) + val rdiff = rhs.filterNot(e => common.exists(e.semanticEquals)) + if (ldiff.isEmpty || rdiff.isEmpty) { + // (a && b) || (a && b && c && ...) => a && b + common.reduce(And) } else { - val ldiff = lhs.filterNot(e => common.exists(e.semanticEquals(_))) - val rdiff = rhs.filterNot(e => common.exists(e.semanticEquals(_))) - if (ldiff.isEmpty || rdiff.isEmpty) { - // (a && b) || (a && b && c && ...) => a && b - common.reduce(And) - } else { - // (a && b && c && ...) || (a && b && d && ...) => - // ((c && ...) || (d && ...)) && a && b - (common :+ Or(ldiff.reduce(And), rdiff.reduce(And))).reduce(And) - } + // (a && b && c && ...) || (a && b && d && ...) => + // ((c && ...) || (d && ...)) && a && b + (common :+ Or(ldiff.reduce(And), rdiff.reduce(And))).reduce(And) } - } // end of Or(left, right) - - case not @ Not(exp) => exp match { - // not(true) => false - case Literal(true, BooleanType) => Literal(false) - // not(false) => true - case Literal(false, BooleanType) => Literal(true) - // not(l > r) => l <= r - case GreaterThan(l, r) => LessThanOrEqual(l, r) - // not(l >= r) => l < r - case GreaterThanOrEqual(l, r) => LessThan(l, r) - // not(l < r) => l >= r - case LessThan(l, r) => GreaterThanOrEqual(l, r) - // not(l <= r) => l > r - case LessThanOrEqual(l, r) => GreaterThan(l, r) - // not(l || r) => not(l) && not(r) - case Or(l, r) => And(Not(l), Not(r)) - // not(l && r) => not(l) or not(r) - case And(l, r) => Or(Not(l), Not(r)) - // not(not(e)) => e - case Not(e) => e - case _ => not - } // end of Not(exp) - - // if (true) a else b => a - // if (false) a else b => b - case e @ If(Literal(v, _), trueValue, falseValue) => if (v == true) trueValue else falseValue + } + + case Not(TrueLiteral) => FalseLiteral + case Not(FalseLiteral) => TrueLiteral + + case Not(a GreaterThan b) => LessThanOrEqual(a, b) + case Not(a GreaterThanOrEqual b) => LessThan(a, b) + + case Not(a LessThan b) => GreaterThanOrEqual(a, b) + case Not(a LessThanOrEqual b) => GreaterThan(a, b) + + case Not(a Or b) => And(Not(a), Not(b)) + case Not(a And b) => Or(Not(a), Not(b)) + + case Not(Not(e)) => e + + case If(TrueLiteral, trueValue, _) => trueValue + case If(FalseLiteral, _, falseValue) => falseValue } } } From aaa2c3b628319178ca1f3f68966ff253c2de49cb Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 11 Jan 2016 19:59:15 -0800 Subject: [PATCH 072/110] [SPARK-11823] Ignores HiveThriftBinaryServerSuite's test jdbc cancel https://issues.apache.org/jira/browse/SPARK-11823 This test often hangs and times out, leaving hanging processes. Let's ignore it for now and improve the test. Author: Yin Huai Closes #10715 from yhuai/SPARK-11823-ignore. --- .../spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index e598284ab22f8..ba3b26e1b7d49 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -347,7 +347,9 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { ) } - test("test jdbc cancel") { + // This test often hangs and then times out, leaving the hanging processes. + // Let's ignore it and improve the test. + ignore("test jdbc cancel") { withJdbcStatement { statement => val queries = Seq( "DROP TABLE IF EXISTS test_map", From 39ae04e6b714e085a1341aa84d8fc5fc827d5f35 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 11 Jan 2016 21:06:22 -0800 Subject: [PATCH 073/110] [SPARK-12692][BUILD][STREAMING] Scala style: Fix the style violation (Space before "," or ":") Fix the style violation (space before , and :). This PR is a followup for #10643. Author: Kousuke Saruta Closes #10685 from sarutak/SPARK-12692-followup-streaming. --- .../clickstream/PageViewGenerator.scala | 14 ++++---- .../spark/streaming/flume/sink/Logging.scala | 8 ++--- .../streaming/flume/FlumeInputDStream.scala | 18 +++++----- .../kafka/DirectKafkaInputDStream.scala | 4 +-- .../streaming/kafka/KafkaInputDStream.scala | 4 +-- .../kafka/ReliableKafkaStreamSuite.scala | 2 +- .../streaming/mqtt/MQTTInputDStream.scala | 4 +-- .../twitter/TwitterInputDStream.scala | 4 +-- project/MimaExcludes.scala | 12 +++++++ .../apache/spark/streaming/Checkpoint.scala | 12 +++---- .../spark/streaming/StreamingContext.scala | 36 +++++++++---------- .../streaming/api/java/JavaDStreamLike.scala | 2 +- .../dstream/ConstantInputDStream.scala | 4 +-- .../dstream/DStreamCheckpointData.scala | 2 +- .../streaming/dstream/FileInputDStream.scala | 18 +++++----- .../streaming/dstream/InputDStream.scala | 6 ++-- .../dstream/PluggableInputDStream.scala | 4 +-- .../streaming/dstream/RawInputDStream.scala | 4 +-- .../dstream/ReceiverInputDStream.scala | 6 ++-- .../dstream/SocketInputDStream.scala | 4 +-- .../streaming/dstream/StateDStream.scala | 6 ++-- .../spark/streaming/receiver/Receiver.scala | 8 ++--- .../streaming/BasicOperationsSuite.scala | 2 +- .../spark/streaming/CheckpointSuite.scala | 2 +- .../spark/streaming/MasterFailureTest.scala | 4 +-- .../spark/streaming/StateMapSuite.scala | 2 +- .../streaming/StreamingContextSuite.scala | 2 +- .../spark/streaming/TestSuiteBase.scala | 4 +-- .../scheduler/ReceiverTrackerSuite.scala | 4 +-- .../streaming/util/WriteAheadLogSuite.scala | 2 +- 30 files changed, 108 insertions(+), 96 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala index ce1a62060ef6c..50216b9bd40f2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala @@ -23,15 +23,15 @@ import java.net.ServerSocket import java.util.Random /** Represents a page view on a website with associated dimension data. */ -class PageView(val url : String, val status : Int, val zipCode : Int, val userID : Int) +class PageView(val url: String, val status: Int, val zipCode: Int, val userID: Int) extends Serializable { - override def toString() : String = { + override def toString(): String = { "%s\t%s\t%s\t%s\n".format(url, status, zipCode, userID) } } object PageView extends Serializable { - def fromString(in : String) : PageView = { + def fromString(in: String): PageView = { val parts = in.split("\t") new PageView(parts(0), parts(1).toInt, parts(2).toInt, parts(3).toInt) } @@ -58,9 +58,9 @@ object PageViewGenerator { 404 -> .05) val userZipCode = Map(94709 -> .5, 94117 -> .5) - val userID = Map((1 to 100).map(_ -> .01) : _*) + val userID = Map((1 to 100).map(_ -> .01): _*) - def pickFromDistribution[T](inputMap : Map[T, Double]) : T = { + def pickFromDistribution[T](inputMap: Map[T, Double]): T = { val rand = new Random().nextDouble() var total = 0.0 for ((item, prob) <- inputMap) { @@ -72,7 +72,7 @@ object PageViewGenerator { inputMap.take(1).head._1 // Shouldn't get here if probabilities add up to 1.0 } - def getNextClickEvent() : String = { + def getNextClickEvent(): String = { val id = pickFromDistribution(userID) val page = pickFromDistribution(pages) val status = pickFromDistribution(httpStatus) @@ -80,7 +80,7 @@ object PageViewGenerator { new PageView(page, status, zipCode, id).toString() } - def main(args : Array[String]) { + def main(args: Array[String]) { if (args.length != 2) { System.err.println("Usage: PageViewGenerator ") System.exit(1) 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 index d87b86932dd41..aa530a7121bd0 100644 --- 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 @@ -26,20 +26,20 @@ import org.slf4j.{Logger, LoggerFactory} 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 + @transient private var _log: Logger = null // Method to get or create the logger for this object protected def log: Logger = { - if (log_ == null) { + 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 = LoggerFactory.getLogger(className) } - log_ + _log } // Log methods that take only a String 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 1bfa35a8b3d1d..74bd0165c6209 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 @@ -41,12 +41,12 @@ import org.apache.spark.util.Utils private[streaming] class FlumeInputDStream[T: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, host: String, port: Int, storageLevel: StorageLevel, enableDecompression: Boolean -) extends ReceiverInputDStream[SparkFlumeEvent](ssc_) { +) extends ReceiverInputDStream[SparkFlumeEvent](_ssc) { override def getReceiver(): Receiver[SparkFlumeEvent] = { new FlumeReceiver(host, port, storageLevel, enableDecompression) @@ -60,7 +60,7 @@ class FlumeInputDStream[T: ClassTag]( * which are not serializable. */ class SparkFlumeEvent() extends Externalizable { - var event : AvroFlumeEvent = new AvroFlumeEvent() + var event: AvroFlumeEvent = new AvroFlumeEvent() /* De-serialize from bytes. */ def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -75,12 +75,12 @@ class SparkFlumeEvent() extends Externalizable { val keyLength = in.readInt() val keyBuff = new Array[Byte](keyLength) in.readFully(keyBuff) - val key : String = Utils.deserialize(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) + val value: String = Utils.deserialize(valBuff) headers.put(key, value) } @@ -109,7 +109,7 @@ class SparkFlumeEvent() extends Externalizable { } private[streaming] object SparkFlumeEvent { - def fromAvroFlumeEvent(in : AvroFlumeEvent) : SparkFlumeEvent = { + def fromAvroFlumeEvent(in: AvroFlumeEvent): SparkFlumeEvent = { val event = new SparkFlumeEvent event.event = in event @@ -118,13 +118,13 @@ private[streaming] object SparkFlumeEvent { /** A simple server that implements Flume's Avro protocol. */ private[streaming] -class FlumeEventServer(receiver : FlumeReceiver) extends AvroSourceProtocol { - override def append(event : AvroFlumeEvent) : Status = { +class FlumeEventServer(receiver: FlumeReceiver) extends AvroSourceProtocol { + override def append(event: AvroFlumeEvent): Status = { receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event)) Status.OK } - override def appendBatch(events : java.util.List[AvroFlumeEvent]) : Status = { + override def appendBatch(events: java.util.List[AvroFlumeEvent]): Status = { events.asScala.foreach(event => receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event))) Status.OK } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 8a087474d3169..54d8c8b03f206 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -58,11 +58,11 @@ class DirectKafkaInputDStream[ U <: Decoder[K]: ClassTag, T <: Decoder[V]: ClassTag, R: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, val kafkaParams: Map[String, String], val fromOffsets: Map[TopicAndPartition, Long], messageHandler: MessageAndMetadata[K, V] => R - ) extends InputDStream[R](ssc_) with Logging { + ) extends InputDStream[R](_ssc) with Logging { val maxRetries = context.sparkContext.getConf.getInt( "spark.streaming.kafka.maxRetries", 1) 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 67f2360896b16..89d1811c99971 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 @@ -48,12 +48,12 @@ class KafkaInputDStream[ V: ClassTag, U <: Decoder[_]: ClassTag, T <: Decoder[_]: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, kafkaParams: Map[String, String], topics: Map[String, Int], useReliableReceiver: Boolean, storageLevel: StorageLevel - ) extends ReceiverInputDStream[(K, V)](ssc_) with Logging { + ) extends ReceiverInputDStream[(K, V)](_ssc) with Logging { def getReceiver(): Receiver[(K, V)] = { if (!useReliableReceiver) { diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index 80e2df62de3fe..7b9aee39ffb76 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -50,7 +50,7 @@ class ReliableKafkaStreamSuite extends SparkFunSuite private var ssc: StreamingContext = _ private var tempDirectory: File = null - override def beforeAll() : Unit = { + override def beforeAll(): Unit = { kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala index 116c170489e96..079bd8a9a87ea 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala @@ -38,11 +38,11 @@ import org.apache.spark.streaming.receiver.Receiver private[streaming] class MQTTInputDStream( - ssc_ : StreamingContext, + _ssc: StreamingContext, brokerUrl: String, topic: String, storageLevel: StorageLevel - ) extends ReceiverInputDStream[String](ssc_) { + ) extends ReceiverInputDStream[String](_ssc) { private[streaming] override def name: String = s"MQTT stream [$id]" 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 a48eec70b9f78..bdd57fdde3b89 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 @@ -39,11 +39,11 @@ import org.apache.spark.streaming.receiver.Receiver */ private[streaming] class TwitterInputDStream( - ssc_ : StreamingContext, + _ssc: StreamingContext, twitterAuth: Option[Authorization], filters: Seq[String], storageLevel: StorageLevel - ) extends ReceiverInputDStream[Status](ssc_) { + ) extends ReceiverInputDStream[Status](_ssc) { private def createOAuthAuthorization(): Authorization = { new OAuthAuthorization(new ConfigurationBuilder().build()) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 0d5f938d9ef5c..4206d1fada421 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -135,6 +135,18 @@ object MimaExcludes { ) ++ Seq( // SPARK-12510 Refactor ActorReceiver to support Java ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver") + ) ++ Seq( + // SPARK-12692 Scala style: Fix the style violation (Space before "," or ":") + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkSink.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkSink.org$apache$spark$streaming$flume$sink$Logging$$log__="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler.org$apache$spark$streaming$flume$sink$Logging$$log__="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$log__="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$_log"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$_log_="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log__=") ) case v if v.startsWith("1.6") => Seq( 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 86f01d2168729..298cdc05acfa9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -183,7 +183,7 @@ class CheckpointWriter( val executor = Executors.newFixedThreadPool(1) val compressionCodec = CompressionCodec.createCodec(conf) private var stopped = false - private var fs_ : FileSystem = _ + private var _fs: FileSystem = _ @volatile private var latestCheckpointTime: Time = null @@ -298,12 +298,12 @@ class CheckpointWriter( } private def fs = synchronized { - if (fs_ == null) fs_ = new Path(checkpointDir).getFileSystem(hadoopConf) - fs_ + if (_fs == null) _fs = new Path(checkpointDir).getFileSystem(hadoopConf) + _fs } private def reset() = synchronized { - fs_ = null + _fs = null } } @@ -370,8 +370,8 @@ object CheckpointReader extends Logging { } private[streaming] -class ObjectInputStreamWithLoader(inputStream_ : InputStream, loader: ClassLoader) - extends ObjectInputStream(inputStream_) { +class ObjectInputStreamWithLoader(_inputStream: InputStream, loader: ClassLoader) + extends ObjectInputStream(_inputStream) { override def resolveClass(desc: ObjectStreamClass): Class[_] = { try { 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 ba509a1030af7..157ee92fd71b3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -58,9 +58,9 @@ import org.apache.spark.util.{AsynchronousListenerBus, CallSite, ShutdownHookMan * of the context by `stop()` or by an exception. */ class StreamingContext private[streaming] ( - sc_ : SparkContext, - cp_ : Checkpoint, - batchDur_ : Duration + _sc: SparkContext, + _cp: Checkpoint, + _batchDur: Duration ) extends Logging { /** @@ -126,18 +126,18 @@ class StreamingContext private[streaming] ( } - if (sc_ == null && cp_ == null) { + if (_sc == null && _cp == null) { throw new Exception("Spark Streaming cannot be initialized with " + "both SparkContext and checkpoint as null") } - private[streaming] val isCheckpointPresent = (cp_ != null) + private[streaming] val isCheckpointPresent = (_cp != null) private[streaming] val sc: SparkContext = { - if (sc_ != null) { - sc_ + if (_sc != null) { + _sc } else if (isCheckpointPresent) { - SparkContext.getOrCreate(cp_.createSparkConf()) + SparkContext.getOrCreate(_cp.createSparkConf()) } else { throw new SparkException("Cannot create StreamingContext without a SparkContext") } @@ -154,13 +154,13 @@ class StreamingContext private[streaming] ( private[streaming] val graph: DStreamGraph = { if (isCheckpointPresent) { - cp_.graph.setContext(this) - cp_.graph.restoreCheckpointData() - cp_.graph + _cp.graph.setContext(this) + _cp.graph.restoreCheckpointData() + _cp.graph } else { - require(batchDur_ != null, "Batch duration for StreamingContext cannot be null") + require(_batchDur != null, "Batch duration for StreamingContext cannot be null") val newGraph = new DStreamGraph() - newGraph.setBatchDuration(batchDur_) + newGraph.setBatchDuration(_batchDur) newGraph } } @@ -169,15 +169,15 @@ class StreamingContext private[streaming] ( private[streaming] var checkpointDir: String = { if (isCheckpointPresent) { - sc.setCheckpointDir(cp_.checkpointDir) - cp_.checkpointDir + sc.setCheckpointDir(_cp.checkpointDir) + _cp.checkpointDir } else { null } } private[streaming] val checkpointDuration: Duration = { - if (isCheckpointPresent) cp_.checkpointDuration else graph.batchDuration + if (isCheckpointPresent) _cp.checkpointDuration else graph.batchDuration } private[streaming] val scheduler = new JobScheduler(this) @@ -246,7 +246,7 @@ class StreamingContext private[streaming] ( } private[streaming] def initialCheckpoint: Checkpoint = { - if (isCheckpointPresent) cp_ else null + if (isCheckpointPresent) _cp else null } private[streaming] def getNewInputStreamId() = nextInputStreamId.getAndIncrement() @@ -460,7 +460,7 @@ class StreamingContext private[streaming] ( def binaryRecordsStream( directory: String, recordLength: Int): DStream[Array[Byte]] = withNamedScope("binary records stream") { - val conf = sc_.hadoopConfiguration + val conf = _sc.hadoopConfiguration conf.setInt(FixedLengthBinaryInputFormat.RECORD_LENGTH_PROPERTY, recordLength) val br = fileStream[LongWritable, BytesWritable, FixedLengthBinaryInputFormat]( directory, FileInputDStream.defaultFilter: Path => Boolean, newFilesOnly = true, conf) 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 733147f63ea2e..a791a474c673d 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 @@ -101,7 +101,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * 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() */ - def countByWindow(windowDuration: Duration, slideDuration: Duration) : JavaDStream[jl.Long] = { + def countByWindow(windowDuration: Duration, slideDuration: Duration): JavaDStream[jl.Long] = { dstream.countByWindow(windowDuration, slideDuration) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala index 695384deb32d7..b5f86fe7794fc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala @@ -25,8 +25,8 @@ import org.apache.spark.streaming.{StreamingContext, Time} /** * An input stream that always returns the same RDD on each timestep. Useful for testing. */ -class ConstantInputDStream[T: ClassTag](ssc_ : StreamingContext, rdd: RDD[T]) - extends InputDStream[T](ssc_) { +class ConstantInputDStream[T: ClassTag](_ssc: StreamingContext, rdd: RDD[T]) + extends InputDStream[T](_ssc) { require(rdd != null, "parameter rdd null is illegal, which will lead to NPE in the following transformation") 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 3eff174c2b66c..a9ce1131ce0c1 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 @@ -39,7 +39,7 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) // in that batch's checkpoint data @transient private var timeToOldestCheckpointFileTime = new HashMap[Time, Time] - @transient private var fileSystem : FileSystem = null + @transient private var fileSystem: FileSystem = null protected[streaming] def currentCheckpointFiles = data.asInstanceOf[HashMap[Time, String]] /** 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 cb5b1f252e90c..1c2325409b53e 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 @@ -73,13 +73,13 @@ import org.apache.spark.util.{SerializableConfiguration, TimeStampedHashMap, Uti */ private[streaming] class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( - ssc_ : StreamingContext, + _ssc: StreamingContext, directory: String, filter: Path => Boolean = FileInputDStream.defaultFilter, newFilesOnly: Boolean = true, conf: Option[Configuration] = None) (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]) - extends InputDStream[(K, V)](ssc_) { + extends InputDStream[(K, V)](_ssc) { private val serializableConfOpt = conf.map(new SerializableConfiguration(_)) @@ -128,8 +128,8 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( // Timestamp of the last round of finding files @transient private var lastNewFileFindingTime = 0L - @transient private var path_ : Path = null - @transient private var fs_ : FileSystem = null + @transient private var _path: Path = null + @transient private var _fs: FileSystem = null override def start() { } @@ -289,17 +289,17 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( } private def directoryPath: Path = { - if (path_ == null) path_ = new Path(directory) - path_ + if (_path == null) _path = new Path(directory) + _path } private def fs: FileSystem = { - if (fs_ == null) fs_ = directoryPath.getFileSystem(ssc.sparkContext.hadoopConfiguration) - fs_ + if (_fs == null) _fs = directoryPath.getFileSystem(ssc.sparkContext.hadoopConfiguration) + _fs } private def reset() { - fs_ = null + _fs = null } @throws(classOf[IOException]) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala index d60f418e5c4de..76f6230f36226 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala @@ -38,10 +38,10 @@ import org.apache.spark.util.Utils * that requires running a receiver on the worker nodes, use * [[org.apache.spark.streaming.dstream.ReceiverInputDStream]] as the parent class. * - * @param ssc_ Streaming context that will execute this input stream + * @param _ssc Streaming context that will execute this input stream */ -abstract class InputDStream[T: ClassTag] (ssc_ : StreamingContext) - extends DStream[T](ssc_) { +abstract class InputDStream[T: ClassTag] (_ssc: StreamingContext) + extends DStream[T](_ssc) { private[streaming] var lastValidTime: Time = null diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala index 2442e4c01a0c0..e003ddb96c860 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala @@ -24,8 +24,8 @@ import org.apache.spark.streaming.receiver.Receiver private[streaming] class PluggableInputDStream[T: ClassTag]( - ssc_ : StreamingContext, - receiver: Receiver[T]) extends ReceiverInputDStream[T](ssc_) { + _ssc: StreamingContext, + receiver: Receiver[T]) extends ReceiverInputDStream[T](_ssc) { def getReceiver(): Receiver[T] = { receiver diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala index ac73dca05a674..409c565380f06 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala @@ -38,11 +38,11 @@ import org.apache.spark.streaming.receiver.Receiver */ private[streaming] class RawInputDStream[T: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, host: String, port: Int, storageLevel: StorageLevel - ) extends ReceiverInputDStream[T](ssc_ ) with Logging { + ) extends ReceiverInputDStream[T](_ssc) with Logging { def getReceiver(): Receiver[T] = { new RawNetworkReceiver(host, port, storageLevel).asInstanceOf[Receiver[T]] 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 565b137228d00..49d8f14f4c390 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 @@ -35,11 +35,11 @@ import org.apache.spark.streaming.util.WriteAheadLogUtils * define [[getReceiver]] function that gets the receiver object of type * [[org.apache.spark.streaming.receiver.Receiver]] that will be sent * to the workers to receive data. - * @param ssc_ Streaming context that will execute this input stream + * @param _ssc Streaming context that will execute this input stream * @tparam T Class type of the object of this stream */ -abstract class ReceiverInputDStream[T: ClassTag](ssc_ : StreamingContext) - extends InputDStream[T](ssc_) { +abstract class ReceiverInputDStream[T: ClassTag](_ssc: StreamingContext) + extends InputDStream[T](_ssc) { /** * Asynchronously maintains & sends new rate limits to the receiver through the receiver tracker. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala index e70fc87c39d95..441477479167a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala @@ -31,12 +31,12 @@ import org.apache.spark.util.NextIterator private[streaming] class SocketInputDStream[T: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, host: String, port: Int, bytesToObjects: InputStream => Iterator[T], storageLevel: StorageLevel - ) extends ReceiverInputDStream[T](ssc_) { + ) extends ReceiverInputDStream[T](_ssc) { def getReceiver(): Receiver[T] = { new SocketReceiver(host, port, bytesToObjects, storageLevel) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index ebbe139a2cdf8..fedffb23952a4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -31,7 +31,7 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)], partitioner: Partitioner, preservePartitioning: Boolean, - initialRDD : Option[RDD[(K, S)]] + initialRDD: Option[RDD[(K, S)]] ) extends DStream[(K, S)](parent.ssc) { super.persist(StorageLevel.MEMORY_ONLY_SER) @@ -43,7 +43,7 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( override val mustCheckpoint = true private [this] def computeUsingPreviousRDD ( - parentRDD : RDD[(K, V)], prevStateRDD : RDD[(K, S)]) = { + parentRDD: RDD[(K, V)], prevStateRDD: RDD[(K, S)]) = { // Define the function for the mapPartition operation on cogrouped RDD; // first map the cogrouped tuple to tuples of required type, // and then apply the update function @@ -98,7 +98,7 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( // first map the grouped tuple to tuples of required type, // and then apply the update function val updateFuncLocal = updateFunc - val finalFunc = (iterator : Iterator[(K, Iterable[V])]) => { + val finalFunc = (iterator: Iterator[(K, Iterable[V])]) => { updateFuncLocal (iterator.map (tuple => (tuple._1, tuple._2.toSeq, None))) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala index 639f4259e2e73..3376cd557d72f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -108,7 +108,7 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable def onStop() /** Override this to specify a preferred location (hostname). */ - def preferredLocation : Option[String] = None + def preferredLocation: Option[String] = None /** * Store a single item of received data to Spark's memory. @@ -257,11 +257,11 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable private var id: Int = -1 /** Handler object that runs the receiver. This is instantiated lazily in the worker. */ - @transient private var _supervisor : ReceiverSupervisor = null + @transient private var _supervisor: ReceiverSupervisor = null /** Set the ID of the DStream that this receiver is associated with. */ - private[streaming] def setReceiverId(id_ : Int) { - id = id_ + private[streaming] def setReceiverId(_id: Int) { + id = _id } /** Attach Network Receiver executor to this receiver. */ 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 9d296c6d3ef8b..25e7ae8262a5f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -186,7 +186,7 @@ class BasicOperationsSuite extends TestSuiteBase { val output = Seq(1 to 8, 101 to 108, 201 to 208) testOperation( input, - (s: DStream[Int]) => s.union(s.map(_ + 4)) , + (s: DStream[Int]) => s.union(s.map(_ + 4)), output ) } 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 4d04138da01f7..4a6b91fbc745e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -44,7 +44,7 @@ import org.apache.spark.util.{Clock, ManualClock, MutableURLClassLoader, ResetSy * A input stream that records the times of restore() invoked */ private[streaming] -class CheckpointInputDStream(ssc_ : StreamingContext) extends InputDStream[Int](ssc_) { +class CheckpointInputDStream(_ssc: StreamingContext) extends InputDStream[Int](_ssc) { protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData override def start(): Unit = { } override def stop(): Unit = { } 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 4e56dfbd424b0..7bbbdebd9b19f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -200,12 +200,12 @@ object MasterFailureTest extends Logging { * the last expected output is generated. Finally, return */ private def runStreams[T: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, lastExpectedOutput: T, maxTimeToRun: Long ): Seq[T] = { - var ssc = ssc_ + var ssc = _ssc var totalTimeRan = 0L var isLastOutputGenerated = false var isTimedOut = false diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala index da0430e263b5f..7a76cafc9a11c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala @@ -280,7 +280,7 @@ class StateMapSuite extends SparkFunSuite { testSerialization(new KryoSerializer(conf), map, msg) } - private def testSerialization[T : ClassTag]( + private def testSerialization[T: ClassTag]( serializer: Serializer, map: OpenHashMapBasedStateMap[T, T], msg: String): OpenHashMapBasedStateMap[T, T] = { 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 0ae4c45988032..197b3d143995a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -896,7 +896,7 @@ object SlowTestReceiver { package object testPackage extends Assertions { def test() { val conf = new SparkConf().setMaster("local").setAppName("CreationSite test") - val ssc = new StreamingContext(conf , Milliseconds(100)) + val ssc = new StreamingContext(conf, Milliseconds(100)) try { val inputStream = ssc.receiverStream(new TestReceiver) 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 54eff2b214290..239b10894ad2c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -58,8 +58,8 @@ private[streaming] class DummyInputDStream(ssc: StreamingContext) extends InputD * replayable, reliable message queue like Kafka. It requires a sequence as input, and * returns the i_th element at the i_th batch unde manual clock. */ -class TestInputStream[T: ClassTag](ssc_ : StreamingContext, input: Seq[Seq[T]], numPartitions: Int) - extends InputDStream[T](ssc_) { +class TestInputStream[T: ClassTag](_ssc: StreamingContext, input: Seq[Seq[T]], numPartitions: Int) + extends InputDStream[T](_ssc) { def start() {} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala index 3bd8d086abf7f..b67189fbd7f03 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala @@ -107,8 +107,8 @@ class ReceiverTrackerSuite extends TestSuiteBase { } /** An input DStream with for testing rate controlling */ -private[streaming] class RateTestInputDStream(@transient ssc_ : StreamingContext) - extends ReceiverInputDStream[Int](ssc_) { +private[streaming] class RateTestInputDStream(@transient _ssc: StreamingContext) + extends ReceiverInputDStream[Int](_ssc) { override def getReceiver(): Receiver[Int] = new RateTestReceiver(id) 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 b5d6a24ce8dd6..734dd93cda471 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 @@ -154,7 +154,7 @@ abstract class CommonWriteAheadLogTests( // Recover old files and generate a second set of log files val dataToWrite2 = generateRandomData() manualClock.advance(100000) - writeDataUsingWriteAheadLog(testDir, dataToWrite2, closeFileAfterWrite, allowBatching , + writeDataUsingWriteAheadLog(testDir, dataToWrite2, closeFileAfterWrite, allowBatching, manualClock) val logFiles2 = getLogFilesInDirectory(testDir) assert(logFiles2.size > logFiles1.size) From 112abf9100f05be436e449817468c50174712c78 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 11 Jan 2016 21:37:54 -0800 Subject: [PATCH 074/110] [SPARK-12692][BUILD][YARN] Scala style: Fix the style violation (Space before "," or ":") Fix the style violation (space before , and :). This PR is a followup for #10643. Author: Kousuke Saruta Closes #10686 from sarutak/SPARK-12692-followup-yarn. --- .../org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index e286aed9f9781..272f1299e0ea9 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -357,7 +357,7 @@ object YarnSparkHadoopUtil { * * @return The correct OOM Error handler JVM option, platform dependent. */ - def getOutOfMemoryErrorArgument : String = { + def getOutOfMemoryErrorArgument: String = { if (Utils.isWindows) { escapeForShell("-XX:OnOutOfMemoryError=taskkill /F /PID %%%%p") } else { From 8cfa218f4f1b05f4d076ec15dd0a033ad3e4500d Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 12 Jan 2016 00:51:00 -0800 Subject: [PATCH 075/110] [SPARK-12692][BUILD][SQL] Scala style: Fix the style violation (Space before "," or ":") Fix the style violation (space before , and :). This PR is a followup for #10643. Author: Kousuke Saruta Closes #10718 from sarutak/SPARK-12692-followup-sql. --- scalastyle-config.xml | 2 +- .../spark/sql/catalyst/ScalaReflection.scala | 6 ++-- .../apache/spark/sql/catalyst/SqlParser.scala | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 6 ++-- .../catalyst/analysis/FunctionRegistry.scala | 4 +-- .../catalyst/analysis/HiveTypeCoercion.scala | 2 +- .../spark/sql/catalyst/dsl/package.scala | 4 ++- .../catalyst/encoders/ExpressionEncoder.scala | 2 +- .../spark/sql/catalyst/encoders/package.scala | 2 +- .../sql/catalyst/expressions/Expression.scala | 2 +- .../expressions/stringExpressions.scala | 6 ++-- .../plans/logical/basicOperators.scala | 6 ++-- .../sql/catalyst/util/NumberConverter.scala | 2 +- .../apache/spark/sql/types/ArrayType.scala | 2 +- .../org/apache/spark/sql/types/Decimal.scala | 2 ++ .../encoders/EncoderErrorMessageSuite.scala | 2 +- .../encoders/ExpressionEncoderSuite.scala | 6 ++-- .../BooleanSimplificationSuite.scala | 2 +- .../scala/org/apache/spark/sql/Column.scala | 4 ++- .../org/apache/spark/sql/DataFrame.scala | 36 +++++++++---------- .../apache/spark/sql/DataFrameHolder.scala | 2 +- .../spark/sql/DataFrameNaFunctions.scala | 8 ++--- .../apache/spark/sql/DataFrameReader.scala | 6 ++-- .../scala/org/apache/spark/sql/Dataset.scala | 18 +++++----- .../org/apache/spark/sql/GroupedData.scala | 10 +++--- .../org/apache/spark/sql/GroupedDataset.scala | 8 ++--- .../org/apache/spark/sql/SQLContext.scala | 12 +++---- .../org/apache/spark/sql/SQLImplicits.scala | 10 +++--- .../org/apache/spark/sql/api/r/SQLUtils.scala | 2 +- .../apache/spark/sql/execution/Exchange.scala | 2 +- .../spark/sql/execution/Queryable.scala | 2 +- .../aggregate/TypedAggregateExpression.scala | 2 +- .../datasources/SqlNewHadoopRDD.scala | 2 +- .../parquet/CatalystSchemaConverter.scala | 2 +- .../execution/joins/CartesianProduct.scala | 2 +- .../sql/execution/metric/SQLMetrics.scala | 2 +- .../sql/execution/stat/FrequentItems.scala | 4 +-- .../apache/spark/sql/expressions/Window.scala | 8 ++--- .../org/apache/spark/sql/functions.scala | 6 ++-- .../spark/sql/jdbc/AggregatedDialect.scala | 2 +- .../apache/spark/sql/jdbc/JdbcDialects.scala | 10 +++--- .../apache/spark/sql/jdbc/MySQLDialect.scala | 7 ++-- .../spark/sql/DatasetAggregatorSuite.scala | 4 +-- .../apache/spark/sql/DatasetCacheSuite.scala | 2 +- .../org/apache/spark/sql/DatasetSuite.scala | 26 +++++++------- .../datasources/json/JsonSuite.scala | 2 +- .../datasources/parquet/ParquetIOSuite.scala | 2 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +-- .../hive/thriftserver/ReflectionUtils.scala | 2 +- .../hive/thriftserver/SparkSQLCLIDriver.scala | 4 +-- .../spark/sql/hive/HiveInspectors.scala | 8 ++--- .../org/apache/spark/sql/hive/hiveUDFs.scala | 2 +- .../spark/sql/hive/HiveInspectorSuite.scala | 4 +-- .../sql/hive/InsertIntoHiveTableSuite.scala | 4 +-- 54 files changed, 150 insertions(+), 141 deletions(-) diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 2439a1f715aba..b873b627219f2 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -218,7 +218,7 @@ This file is divided into 3 sections: - + COLON, COMMA 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 79f723cf9b8a0..23fea0e2832a1 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 @@ -49,7 +49,7 @@ object ScalaReflection extends ScalaReflection { * Unlike `schemaFor`, this function doesn't do any massaging of types into the Spark SQL type * system. As a result, ObjectType will be returned for things like boxed Integers */ - def dataTypeFor[T : TypeTag]: DataType = dataTypeFor(localTypeOf[T]) + def dataTypeFor[T: TypeTag]: DataType = dataTypeFor(localTypeOf[T]) private def dataTypeFor(tpe: `Type`): DataType = ScalaReflectionLock.synchronized { tpe match { @@ -116,7 +116,7 @@ object ScalaReflection extends ScalaReflection { * from ordinal 0 (since there are no names to map to). The actual location can be moved by * calling resolve/bind with a new schema. */ - def constructorFor[T : TypeTag]: Expression = { + def constructorFor[T: TypeTag]: Expression = { val tpe = localTypeOf[T] val clsName = getClassNameFromType(tpe) val walkedTypePath = s"""- root class: "${clsName}"""" :: Nil @@ -386,7 +386,7 @@ object ScalaReflection extends ScalaReflection { * * the element type of [[Array]] or [[Seq]]: `array element class: "abc.xyz.MyClass"` * * the field of [[Product]]: `field (class: "abc.xyz.MyClass", name: "myField")` */ - def extractorsFor[T : TypeTag](inputObject: Expression): CreateNamedStruct = { + def extractorsFor[T: TypeTag](inputObject: Expression): CreateNamedStruct = { val tpe = localTypeOf[T] val clsName = getClassNameFromType(tpe) val walkedTypePath = s"""- root class: "${clsName}"""" :: Nil 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 2a132d8b82bef..6ec408a673c79 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 @@ -203,7 +203,7 @@ object SqlParser extends AbstractSparkSQLParser with DataTypeParser { ) protected lazy val ordering: Parser[Seq[SortOrder]] = - ( rep1sep(expression ~ direction.? , ",") ^^ { + ( rep1sep(expression ~ direction.?, ",") ^^ { case exps => exps.map(pair => SortOrder(pair._1, pair._2.getOrElse(Ascending))) } ) 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 8a33af8207350..d16880bc4a9c9 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 @@ -84,7 +84,7 @@ class Analyzer( ResolveAggregateFunctions :: DistinctAggregationRewriter(conf) :: HiveTypeCoercion.typeCoercionRules ++ - extendedResolutionRules : _*), + extendedResolutionRules: _*), Batch("Nondeterministic", Once, PullOutNondeterministic), Batch("UDF", Once, @@ -110,7 +110,7 @@ class Analyzer( // Taking into account the reasonableness and the implementation complexity, // here use the CTE definition first, check table name only and ignore database name // see https://github.com/apache/spark/pull/4929#discussion_r27186638 for more info - case u : UnresolvedRelation => + case u: UnresolvedRelation => val substituted = cteRelations.get(u.tableIdentifier.table).map { relation => val withAlias = u.alias.map(Subquery(_, relation)) withAlias.getOrElse(relation) @@ -889,7 +889,7 @@ class Analyzer( _.transform { // Extracts children expressions of a WindowFunction (input parameters of // a WindowFunction). - case wf : WindowFunction => + case wf: WindowFunction => val newChildren = wf.children.map(extractExpr) wf.withNewChildren(newChildren) 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 5c2aa3c06b3e7..7c3d45b1e40c0 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 @@ -323,13 +323,13 @@ object FunctionRegistry { } else { // Otherwise, find an ctor method that matches the number of arguments, and use that. val params = Seq.fill(expressions.size)(classOf[Expression]) - val f = Try(tag.runtimeClass.getDeclaredConstructor(params : _*)) match { + val f = Try(tag.runtimeClass.getDeclaredConstructor(params: _*)) match { case Success(e) => e case Failure(e) => throw new AnalysisException(s"Invalid number of arguments for function $name") } - Try(f.newInstance(expressions : _*).asInstanceOf[Expression]) match { + Try(f.newInstance(expressions: _*).asInstanceOf[Expression]) match { case Success(e) => e case Failure(e) => throw new AnalysisException(e.getMessage) } 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 dbcbd6854b474..e326ea782700c 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 @@ -529,7 +529,7 @@ object HiveTypeCoercion { if falseValues.contains(value) => And(IsNotNull(bool), Not(bool)) case EqualTo(left @ BooleanType(), right @ NumericType()) => - transform(left , right) + transform(left, right) case EqualTo(left @ NumericType(), right @ BooleanType()) => transform(right, left) case EqualNullSafe(left @ BooleanType(), right @ NumericType()) => 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 5ac1984043d87..c4dbcb7b60628 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 @@ -61,9 +61,11 @@ package object dsl { trait ImplicitOperators { def expr: Expression + // scalastyle:off whitespacebeforetoken def unary_- : Expression = UnaryMinus(expr) def unary_! : Predicate = Not(expr) def unary_~ : Expression = BitwiseNot(expr) + // scalastyle:on whitespacebeforetoken def + (other: Expression): Expression = Add(expr, other) def - (other: Expression): Expression = Subtract(expr, other) @@ -141,7 +143,7 @@ package object dsl { // Note that if we make ExpressionConversions an object rather than a trait, we can // then make this a value class to avoid the small penalty of runtime instantiation. def $(args: Any*): analysis.UnresolvedAttribute = { - analysis.UnresolvedAttribute(sc.s(args : _*)) + analysis.UnresolvedAttribute(sc.s(args: _*)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index 05f746e72b498..fa4c2d93eccec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -44,7 +44,7 @@ import org.apache.spark.util.Utils * to the name `value`. */ object ExpressionEncoder { - def apply[T : TypeTag](): ExpressionEncoder[T] = { + def apply[T: TypeTag](): ExpressionEncoder[T] = { // We convert the not-serializable TypeTag into StructType and ClassTag. val mirror = typeTag[T].mirror val cls = mirror.runtimeClass(typeTag[T].tpe) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala index 9e283f5eb6342..08ada1f38ba96 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala @@ -27,7 +27,7 @@ package object encoders { * references from a specific schema.) This requirement allows us to preserve whether a given * object type is being bound by name or by ordinal when doing resolution. */ - private[sql] def encoderFor[A : Encoder]: ExpressionEncoder[A] = implicitly[Encoder[A]] match { + private[sql] def encoderFor[A: Encoder]: ExpressionEncoder[A] = implicitly[Encoder[A]] match { case e: ExpressionEncoder[A] => e.assertUnresolved() e 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 d6219514b752b..4ffbfa57e726d 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 @@ -164,7 +164,7 @@ abstract class Expression extends TreeNode[Expression] { * Returns the hash for this expression. Expressions that compute the same result, even if * they differ cosmetically should return the same hash. */ - def semanticHash() : Int = { + def semanticHash(): Int = { def computeHash(e: Seq[Any]): Int = { // See http://stackoverflow.com/questions/113511/hash-code-implementation var hash: Int = 17 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index 931f752b4dc1a..bf41f85f79096 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -46,7 +46,7 @@ case class Concat(children: Seq[Expression]) extends Expression with ImplicitCas override def eval(input: InternalRow): Any = { val inputs = children.map(_.eval(input).asInstanceOf[UTF8String]) - UTF8String.concat(inputs : _*) + UTF8String.concat(inputs: _*) } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { @@ -99,7 +99,7 @@ case class ConcatWs(children: Seq[Expression]) case null => Iterator(null.asInstanceOf[UTF8String]) } } - UTF8String.concatWs(flatInputs.head, flatInputs.tail : _*) + UTF8String.concatWs(flatInputs.head, flatInputs.tail: _*) } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { @@ -990,7 +990,7 @@ case class FormatNumber(x: Expression, d: Expression) def typeHelper(p: String): String = { x.dataType match { - case _ : DecimalType => s"""$p.toJavaBigDecimal()""" + case _: DecimalType => s"""$p.toJavaBigDecimal()""" case _ => s"$p" } } 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 64957db6b4013..5489051e9501b 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 @@ -496,7 +496,7 @@ case class MapPartitions[T, U]( /** Factory for constructing new `AppendColumn` nodes. */ object AppendColumns { - def apply[T, U : Encoder]( + def apply[T, U: Encoder]( func: T => U, tEncoder: ExpressionEncoder[T], child: LogicalPlan): AppendColumns[T, U] = { @@ -522,7 +522,7 @@ case class AppendColumns[T, U]( /** Factory for constructing new `MapGroups` nodes. */ object MapGroups { - def apply[K, T, U : Encoder]( + def apply[K, T, U: Encoder]( func: (K, Iterator[T]) => TraversableOnce[U], kEncoder: ExpressionEncoder[K], tEncoder: ExpressionEncoder[T], @@ -557,7 +557,7 @@ case class MapGroups[K, T, U]( /** Factory for constructing new `CoGroup` nodes. */ object CoGroup { - def apply[Key, Left, Right, Result : Encoder]( + def apply[Key, Left, Right, Result: Encoder]( func: (Key, Iterator[Left], Iterator[Right]) => TraversableOnce[Result], keyEnc: ExpressionEncoder[Key], leftEnc: ExpressionEncoder[Left], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala index 9fefc5656aac0..e4417e0955143 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala @@ -122,7 +122,7 @@ object NumberConverter { * unsigned, otherwise it is signed. * NB: This logic is borrowed from org.apache.hadoop.hive.ql.ud.UDFConv */ - def convert(n: Array[Byte] , fromBase: Int, toBase: Int ): UTF8String = { + def convert(n: Array[Byte], fromBase: Int, toBase: Int ): UTF8String = { if (fromBase < Character.MIN_RADIX || fromBase > Character.MAX_RADIX || Math.abs(toBase) < Character.MIN_RADIX || Math.abs(toBase) > Character.MAX_RADIX) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala index 520e344361625..be7573b95d841 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala @@ -90,7 +90,7 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT private[sql] lazy val interpretedOrdering: Ordering[ArrayData] = new Ordering[ArrayData] { private[this] val elementOrdering: Ordering[Any] = elementType match { case dt: AtomicType => dt.ordering.asInstanceOf[Ordering[Any]] - case a : ArrayType => a.interpretedOrdering.asInstanceOf[Ordering[Any]] + case a: ArrayType => a.interpretedOrdering.asInstanceOf[Ordering[Any]] case s: StructType => s.interpretedOrdering.asInstanceOf[Ordering[Any]] case other => throw new IllegalArgumentException(s"Type $other does not support ordered operations") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 38ce1604b1ede..864b47a2a08aa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -310,6 +310,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { def remainder(that: Decimal): Decimal = this % that + // scalastyle:off whitespacebeforetoken def unary_- : Decimal = { if (decimalVal.ne(null)) { Decimal(-decimalVal, precision, scale) @@ -317,6 +318,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { Decimal(-longVal, precision, scale) } } + // scalastyle:on whitespacebeforetoken def abs: Decimal = if (this.compare(Decimal.ZERO) < 0) this.unary_- else this diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala index 8c766ef829923..a1c4a861c610f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala @@ -98,5 +98,5 @@ class EncoderErrorMessageSuite extends SparkFunSuite { s"""array element class: "${clsName[NonEncodable]}"""")) } - private def clsName[T : ClassTag]: String = implicitly[ClassTag[T]].runtimeClass.getName + private def clsName[T: ClassTag]: String = implicitly[ClassTag[T]].runtimeClass.getName } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 88c558d80a79a..67f4dc98be231 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -80,7 +80,7 @@ class JavaSerializable(val value: Int) extends Serializable { class ExpressionEncoderSuite extends SparkFunSuite { OuterScopes.outerScopes.put(getClass.getName, this) - implicit def encoder[T : TypeTag]: ExpressionEncoder[T] = ExpressionEncoder() + implicit def encoder[T: TypeTag]: ExpressionEncoder[T] = ExpressionEncoder() // test flat encoders encodeDecodeTest(false, "primitive boolean") @@ -145,7 +145,7 @@ class ExpressionEncoderSuite extends SparkFunSuite { encoderFor(Encoders.javaSerialization[JavaSerializable])) // test product encoders - private def productTest[T <: Product : ExpressionEncoder](input: T): Unit = { + private def productTest[T <: Product: ExpressionEncoder](input: T): Unit = { encodeDecodeTest(input, input.getClass.getSimpleName) } @@ -286,7 +286,7 @@ class ExpressionEncoderSuite extends SparkFunSuite { } } - private def encodeDecodeTest[T : ExpressionEncoder]( + private def encodeDecodeTest[T: ExpressionEncoder]( input: T, testName: String): Unit = { test(s"encode/decode for $testName: $input") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index 000a3b7ecb7c6..6932f185b9d62 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -80,7 +80,7 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { checkCondition(('a < 2 || 'a > 3 || 'b > 5) && 'a < 2, 'a < 2) - checkCondition('a < 2 && ('a < 2 || 'a > 3 || 'b > 5) , 'a < 2) + checkCondition('a < 2 && ('a < 2 || 'a > 3 || 'b > 5), 'a < 2) checkCondition(('a < 2 || 'b > 3) && ('a < 2 || 'c > 5), 'a < 2 || ('b > 3 && 'c > 5)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index e8c61d6e01dc3..a434d03332459 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -152,7 +152,7 @@ class Column(protected[sql] val expr: Expression) extends Logging { * results into the correct JVM types. * @since 1.6.0 */ - def as[U : Encoder]: TypedColumn[Any, U] = new TypedColumn[Any, U](expr, encoderFor[U]) + def as[U: Encoder]: TypedColumn[Any, U] = new TypedColumn[Any, U](expr, encoderFor[U]) /** * Extracts a value or values from a complex type. @@ -171,6 +171,7 @@ class Column(protected[sql] val expr: Expression) extends Logging { UnresolvedExtractValue(expr, lit(extraction).expr) } + // scalastyle:off whitespacebeforetoken /** * Unary minus, i.e. negate the expression. * {{{ @@ -202,6 +203,7 @@ class Column(protected[sql] val expr: Expression) extends Logging { * @since 1.3.0 */ def unary_! : Column = withExpr { Not(expr) } + // scalastyle:on whitespacebeforetoken /** * Equality test. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 60d2f05b8605b..fac8950aee12d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -204,7 +204,7 @@ class DataFrame private[sql]( * @since 1.6.0 */ @Experimental - def as[U : Encoder]: Dataset[U] = new Dataset[U](sqlContext, logicalPlan) + def as[U: Encoder]: Dataset[U] = new Dataset[U](sqlContext, logicalPlan) /** * Returns a new [[DataFrame]] with columns renamed. This can be quite convenient in conversion @@ -227,7 +227,7 @@ class DataFrame private[sql]( val newCols = logicalPlan.output.zip(colNames).map { case (oldAttribute, newName) => Column(oldAttribute).as(newName) } - select(newCols : _*) + select(newCols: _*) } /** @@ -579,7 +579,7 @@ class DataFrame private[sql]( */ @scala.annotation.varargs def sortWithinPartitions(sortCol: String, sortCols: String*): DataFrame = { - sortWithinPartitions((sortCol +: sortCols).map(Column(_)) : _*) + sortWithinPartitions((sortCol +: sortCols).map(Column(_)): _*) } /** @@ -608,7 +608,7 @@ class DataFrame private[sql]( */ @scala.annotation.varargs def sort(sortCol: String, sortCols: String*): DataFrame = { - sort((sortCol +: sortCols).map(apply) : _*) + sort((sortCol +: sortCols).map(apply): _*) } /** @@ -631,7 +631,7 @@ class DataFrame private[sql]( * @since 1.3.0 */ @scala.annotation.varargs - def orderBy(sortCol: String, sortCols: String*): DataFrame = sort(sortCol, sortCols : _*) + def orderBy(sortCol: String, sortCols: String*): DataFrame = sort(sortCol, sortCols: _*) /** * Returns a new [[DataFrame]] sorted by the given expressions. @@ -640,7 +640,7 @@ class DataFrame private[sql]( * @since 1.3.0 */ @scala.annotation.varargs - def orderBy(sortExprs: Column*): DataFrame = sort(sortExprs : _*) + def orderBy(sortExprs: Column*): DataFrame = sort(sortExprs: _*) /** * Selects column based on the column name and return it as a [[Column]]. @@ -720,7 +720,7 @@ class DataFrame private[sql]( * @since 1.3.0 */ @scala.annotation.varargs - def select(col: String, cols: String*): DataFrame = select((col +: cols).map(Column(_)) : _*) + def select(col: String, cols: String*): DataFrame = select((col +: cols).map(Column(_)): _*) /** * Selects a set of SQL expressions. This is a variant of `select` that accepts @@ -948,7 +948,7 @@ class DataFrame private[sql]( * @since 1.3.0 */ def agg(aggExpr: (String, String), aggExprs: (String, String)*): DataFrame = { - groupBy().agg(aggExpr, aggExprs : _*) + groupBy().agg(aggExpr, aggExprs: _*) } /** @@ -986,7 +986,7 @@ class DataFrame private[sql]( * @since 1.3.0 */ @scala.annotation.varargs - def agg(expr: Column, exprs: Column*): DataFrame = groupBy().agg(expr, exprs : _*) + def agg(expr: Column, exprs: Column*): DataFrame = groupBy().agg(expr, exprs: _*) /** * Returns a new [[DataFrame]] by taking the first `n` rows. The difference between this function @@ -1118,7 +1118,7 @@ class DataFrame private[sql]( * @group dfops * @since 1.3.0 */ - def explode[A <: Product : TypeTag](input: Column*)(f: Row => TraversableOnce[A]): DataFrame = { + def explode[A <: Product: TypeTag](input: Column*)(f: Row => TraversableOnce[A]): DataFrame = { val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val elementTypes = schema.toAttributes.map { @@ -1147,7 +1147,7 @@ class DataFrame private[sql]( * @group dfops * @since 1.3.0 */ - def explode[A, B : TypeTag](inputColumn: String, outputColumn: String)(f: A => TraversableOnce[B]) + def explode[A, B: TypeTag](inputColumn: String, outputColumn: String)(f: A => TraversableOnce[B]) : DataFrame = { val dataType = ScalaReflection.schemaFor[B].dataType val attributes = AttributeReference(outputColumn, dataType)() :: Nil @@ -1186,7 +1186,7 @@ class DataFrame private[sql]( Column(field) } } - select(columns : _*) + select(columns: _*) } else { select(Column("*"), col.as(colName)) } @@ -1207,7 +1207,7 @@ class DataFrame private[sql]( Column(field) } } - select(columns : _*) + select(columns: _*) } else { select(Column("*"), col.as(colName, metadata)) } @@ -1231,7 +1231,7 @@ class DataFrame private[sql]( Column(col) } } - select(columns : _*) + select(columns: _*) } else { this } @@ -1244,7 +1244,7 @@ class DataFrame private[sql]( * @since 1.4.0 */ def drop(colName: String): DataFrame = { - drop(Seq(colName) : _*) + drop(Seq(colName): _*) } /** @@ -1283,7 +1283,7 @@ class DataFrame private[sql]( val colsAfterDrop = attrs.filter { attr => attr != expression }.map(attr => Column(attr)) - select(colsAfterDrop : _*) + select(colsAfterDrop: _*) } /** @@ -1479,7 +1479,7 @@ class DataFrame private[sql]( * @group action * @since 1.6.0 */ - def takeAsList(n: Int): java.util.List[Row] = java.util.Arrays.asList(take(n) : _*) + def takeAsList(n: Int): java.util.List[Row] = java.util.Arrays.asList(take(n): _*) /** * Returns an array that contains all of [[Row]]s in this [[DataFrame]]. @@ -1505,7 +1505,7 @@ class DataFrame private[sql]( */ def collectAsList(): java.util.List[Row] = withCallback("collectAsList", this) { _ => withNewExecutionId { - java.util.Arrays.asList(rdd.collect() : _*) + java.util.Arrays.asList(rdd.collect(): _*) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameHolder.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameHolder.scala index 3b30337f1f877..4441a634be407 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameHolder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameHolder.scala @@ -33,5 +33,5 @@ case class DataFrameHolder private[sql](private val df: DataFrame) { // `rdd.toDF("1")` as invoking this toDF and then apply on the returned DataFrame. def toDF(): DataFrame = df - def toDF(colNames: String*): DataFrame = df.toDF(colNames : _*) + def toDF(colNames: String*): DataFrame = df.toDF(colNames: _*) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index f7be5f6b370ab..43500b09e0f38 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -164,7 +164,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { df.col(f.name) } } - df.select(projections : _*) + df.select(projections: _*) } /** @@ -191,7 +191,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { df.col(f.name) } } - df.select(projections : _*) + df.select(projections: _*) } /** @@ -364,7 +364,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { df.col(f.name) } } - df.select(projections : _*) + df.select(projections: _*) } private def fill0(values: Seq[(String, Any)]): DataFrame = { @@ -395,7 +395,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { } }.getOrElse(df.col(f.name)) } - df.select(projections : _*) + df.select(projections: _*) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index d948e4894253c..1ed451d5a8bab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -203,7 +203,7 @@ class DataFrameReader private[sql](sqlContext: SQLContext) extends Logging { predicates: Array[String], connectionProperties: Properties): DataFrame = { val parts: Array[Partition] = predicates.zipWithIndex.map { case (part, i) => - JDBCPartition(part, i) : Partition + JDBCPartition(part, i): Partition } jdbc(url, table, parts, connectionProperties) } @@ -262,7 +262,7 @@ class DataFrameReader private[sql](sqlContext: SQLContext) extends Logging { * * @since 1.6.0 */ - def json(paths: String*): DataFrame = format("json").load(paths : _*) + def json(paths: String*): DataFrame = format("json").load(paths: _*) /** * Loads an `JavaRDD[String]` storing JSON objects (one object per record) and @@ -355,7 +355,7 @@ class DataFrameReader private[sql](sqlContext: SQLContext) extends Logging { * @since 1.6.0 */ @scala.annotation.varargs - def text(paths: String*): DataFrame = format("text").load(paths : _*) + def text(paths: String*): DataFrame = format("text").load(paths: _*) /////////////////////////////////////////////////////////////////////////////////////// // Builder pattern config options diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 42f01e9359c64..9ffb5b94b2d18 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -131,7 +131,7 @@ class Dataset[T] private[sql]( * along with `alias` or `as` to rearrange or rename as required. * @since 1.6.0 */ - def as[U : Encoder]: Dataset[U] = { + def as[U: Encoder]: Dataset[U] = { new Dataset(sqlContext, queryExecution, encoderFor[U]) } @@ -318,7 +318,7 @@ class Dataset[T] private[sql]( * Returns a new [[Dataset]] that contains the result of applying `func` to each element. * @since 1.6.0 */ - def map[U : Encoder](func: T => U): Dataset[U] = mapPartitions(_.map(func)) + def map[U: Encoder](func: T => U): Dataset[U] = mapPartitions(_.map(func)) /** * (Java-specific) @@ -333,7 +333,7 @@ class Dataset[T] private[sql]( * Returns a new [[Dataset]] that contains the result of applying `func` to each partition. * @since 1.6.0 */ - def mapPartitions[U : Encoder](func: Iterator[T] => Iterator[U]): Dataset[U] = { + def mapPartitions[U: Encoder](func: Iterator[T] => Iterator[U]): Dataset[U] = { new Dataset[U]( sqlContext, MapPartitions[T, U]( @@ -360,7 +360,7 @@ class Dataset[T] private[sql]( * and then flattening the results. * @since 1.6.0 */ - def flatMap[U : Encoder](func: T => TraversableOnce[U]): Dataset[U] = + def flatMap[U: Encoder](func: T => TraversableOnce[U]): Dataset[U] = mapPartitions(_.flatMap(func)) /** @@ -432,7 +432,7 @@ class Dataset[T] private[sql]( * Returns a [[GroupedDataset]] where the data is grouped by the given key `func`. * @since 1.6.0 */ - def groupBy[K : Encoder](func: T => K): GroupedDataset[K, T] = { + def groupBy[K: Encoder](func: T => K): GroupedDataset[K, T] = { val inputPlan = logicalPlan val withGroupingKey = AppendColumns(func, resolvedTEncoder, inputPlan) val executed = sqlContext.executePlan(withGroupingKey) @@ -566,14 +566,14 @@ class Dataset[T] private[sql]( * Returns a new [[Dataset]] by sampling a fraction of records. * @since 1.6.0 */ - def sample(withReplacement: Boolean, fraction: Double, seed: Long) : Dataset[T] = + def sample(withReplacement: Boolean, fraction: Double, seed: Long): Dataset[T] = withPlan(Sample(0.0, fraction, withReplacement, seed, _)) /** * Returns a new [[Dataset]] by sampling a fraction of records, using a random seed. * @since 1.6.0 */ - def sample(withReplacement: Boolean, fraction: Double) : Dataset[T] = { + def sample(withReplacement: Boolean, fraction: Double): Dataset[T] = { sample(withReplacement, fraction, Utils.random.nextLong) } @@ -731,7 +731,7 @@ class Dataset[T] private[sql]( * a very large `num` can crash the driver process with OutOfMemoryError. * @since 1.6.0 */ - def takeAsList(num: Int): java.util.List[T] = java.util.Arrays.asList(take(num) : _*) + def takeAsList(num: Int): java.util.List[T] = java.util.Arrays.asList(take(num): _*) /** * Persist this [[Dataset]] with the default storage level (`MEMORY_AND_DISK`). @@ -786,7 +786,7 @@ class Dataset[T] private[sql]( private[sql] def withPlan(f: LogicalPlan => LogicalPlan): Dataset[T] = new Dataset[T](sqlContext, sqlContext.executePlan(f(logicalPlan)), tEncoder) - private[sql] def withPlan[R : Encoder]( + private[sql] def withPlan[R: Encoder]( other: Dataset[_])( f: (LogicalPlan, LogicalPlan) => LogicalPlan): Dataset[R] = new Dataset[R](sqlContext, f(logicalPlan, other.logicalPlan)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala index c74ef2c03541e..f5cbf013bce9d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala @@ -229,7 +229,7 @@ class GroupedData protected[sql]( */ @scala.annotation.varargs def mean(colNames: String*): DataFrame = { - aggregateNumericColumns(colNames : _*)(Average) + aggregateNumericColumns(colNames: _*)(Average) } /** @@ -241,7 +241,7 @@ class GroupedData protected[sql]( */ @scala.annotation.varargs def max(colNames: String*): DataFrame = { - aggregateNumericColumns(colNames : _*)(Max) + aggregateNumericColumns(colNames: _*)(Max) } /** @@ -253,7 +253,7 @@ class GroupedData protected[sql]( */ @scala.annotation.varargs def avg(colNames: String*): DataFrame = { - aggregateNumericColumns(colNames : _*)(Average) + aggregateNumericColumns(colNames: _*)(Average) } /** @@ -265,7 +265,7 @@ class GroupedData protected[sql]( */ @scala.annotation.varargs def min(colNames: String*): DataFrame = { - aggregateNumericColumns(colNames : _*)(Min) + aggregateNumericColumns(colNames: _*)(Min) } /** @@ -277,7 +277,7 @@ class GroupedData protected[sql]( */ @scala.annotation.varargs def sum(colNames: String*): DataFrame = { - aggregateNumericColumns(colNames : _*)(Sum) + aggregateNumericColumns(colNames: _*)(Sum) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala index a819ddceb1b1b..12179367fa012 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala @@ -73,7 +73,7 @@ class GroupedDataset[K, V] private[sql]( * * @since 1.6.0 */ - def keyAs[L : Encoder]: GroupedDataset[L, V] = + def keyAs[L: Encoder]: GroupedDataset[L, V] = new GroupedDataset( encoderFor[L], unresolvedVEncoder, @@ -110,7 +110,7 @@ class GroupedDataset[K, V] private[sql]( * * @since 1.6.0 */ - def flatMapGroups[U : Encoder](f: (K, Iterator[V]) => TraversableOnce[U]): Dataset[U] = { + def flatMapGroups[U: Encoder](f: (K, Iterator[V]) => TraversableOnce[U]): Dataset[U] = { new Dataset[U]( sqlContext, MapGroups( @@ -158,7 +158,7 @@ class GroupedDataset[K, V] private[sql]( * * @since 1.6.0 */ - def mapGroups[U : Encoder](f: (K, Iterator[V]) => U): Dataset[U] = { + def mapGroups[U: Encoder](f: (K, Iterator[V]) => U): Dataset[U] = { val func = (key: K, it: Iterator[V]) => Iterator(f(key, it)) flatMapGroups(func) } @@ -302,7 +302,7 @@ class GroupedDataset[K, V] private[sql]( * * @since 1.6.0 */ - def cogroup[U, R : Encoder]( + def cogroup[U, R: Encoder]( other: GroupedDataset[K, U])( f: (K, Iterator[V], Iterator[U]) => TraversableOnce[R]): Dataset[R] = { new Dataset[R]( 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 e827427c19e25..61c74f83409e9 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 @@ -409,7 +409,7 @@ class SQLContext private[sql]( * @since 1.3.0 */ @Experimental - def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { + def createDataFrame[A <: Product: TypeTag](rdd: RDD[A]): DataFrame = { SQLContext.setActive(self) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val attributeSeq = schema.toAttributes @@ -425,7 +425,7 @@ class SQLContext private[sql]( * @since 1.3.0 */ @Experimental - def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { + def createDataFrame[A <: Product: TypeTag](data: Seq[A]): DataFrame = { SQLContext.setActive(self) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val attributeSeq = schema.toAttributes @@ -498,7 +498,7 @@ class SQLContext private[sql]( } - def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = { + def createDataset[T: Encoder](data: Seq[T]): Dataset[T] = { val enc = encoderFor[T] val attributes = enc.schema.toAttributes val encoded = data.map(d => enc.toRow(d).copy()) @@ -507,7 +507,7 @@ class SQLContext private[sql]( new Dataset[T](this, plan) } - def createDataset[T : Encoder](data: RDD[T]): Dataset[T] = { + def createDataset[T: Encoder](data: RDD[T]): Dataset[T] = { val enc = encoderFor[T] val attributes = enc.schema.toAttributes val encoded = data.map(d => enc.toRow(d)) @@ -516,7 +516,7 @@ class SQLContext private[sql]( new Dataset[T](this, plan) } - def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = { + def createDataset[T: Encoder](data: java.util.List[T]): Dataset[T] = { createDataset(data.asScala) } @@ -945,7 +945,7 @@ class SQLContext private[sql]( } } - // Register a succesfully instantiatd context to the singleton. This should be at the end of + // Register a successfully instantiated context to the singleton. This should be at the end of // the class definition so that the singleton is updated only if there is no exception in the // construction of the instance. sparkContext.addSparkListener(new SparkListener { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala index ab414799f1a42..a7f7997df1a8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala @@ -37,7 +37,7 @@ abstract class SQLImplicits { protected def _sqlContext: SQLContext /** @since 1.6.0 */ - implicit def newProductEncoder[T <: Product : TypeTag]: Encoder[T] = ExpressionEncoder() + implicit def newProductEncoder[T <: Product: TypeTag]: Encoder[T] = ExpressionEncoder() /** @since 1.6.0 */ implicit def newIntEncoder: Encoder[Int] = ExpressionEncoder() @@ -67,7 +67,7 @@ abstract class SQLImplicits { * Creates a [[Dataset]] from an RDD. * @since 1.6.0 */ - implicit def rddToDatasetHolder[T : Encoder](rdd: RDD[T]): DatasetHolder[T] = { + implicit def rddToDatasetHolder[T: Encoder](rdd: RDD[T]): DatasetHolder[T] = { DatasetHolder(_sqlContext.createDataset(rdd)) } @@ -75,7 +75,7 @@ abstract class SQLImplicits { * Creates a [[Dataset]] from a local Seq. * @since 1.6.0 */ - implicit def localSeqToDatasetHolder[T : Encoder](s: Seq[T]): DatasetHolder[T] = { + implicit def localSeqToDatasetHolder[T: Encoder](s: Seq[T]): DatasetHolder[T] = { DatasetHolder(_sqlContext.createDataset(s)) } @@ -89,7 +89,7 @@ abstract class SQLImplicits { * Creates a DataFrame from an RDD of Product (e.g. case classes, tuples). * @since 1.3.0 */ - implicit def rddToDataFrameHolder[A <: Product : TypeTag](rdd: RDD[A]): DataFrameHolder = { + implicit def rddToDataFrameHolder[A <: Product: TypeTag](rdd: RDD[A]): DataFrameHolder = { DataFrameHolder(_sqlContext.createDataFrame(rdd)) } @@ -97,7 +97,7 @@ abstract class SQLImplicits { * Creates a DataFrame from a local Seq of Product. * @since 1.3.0 */ - implicit def localSeqToDataFrameHolder[A <: Product : TypeTag](data: Seq[A]): DataFrameHolder = + implicit def localSeqToDataFrameHolder[A <: Product: TypeTag](data: Seq[A]): DataFrameHolder = { DataFrameHolder(_sqlContext.createDataFrame(data)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index d912aeb70d517..a8e6a40169d81 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -39,7 +39,7 @@ private[r] object SQLUtils { new JavaSparkContext(sqlCtx.sparkContext) } - def createStructType(fields : Seq[StructField]): StructType = { + def createStructType(fields: Seq[StructField]): StructType = { StructType(fields) } 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 6b100577077c6..058d147c7d65d 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 @@ -223,7 +223,7 @@ case class Exchange( new ShuffledRowRDD(shuffleDependency, specifiedPartitionStartIndices) } - protected override def doExecute(): RDD[InternalRow] = attachTree(this , "execute") { + protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") { coordinator match { case Some(exchangeCoordinator) => val shuffleRDD = exchangeCoordinator.postShuffleRDD(this) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Queryable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Queryable.scala index 38263af0f7e30..bb551614779b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Queryable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Queryable.scala @@ -71,7 +71,7 @@ private[sql] trait Queryable { private[sql] def formatString ( rows: Seq[Seq[String]], numRows: Int, - hasMoreData : Boolean, + hasMoreData: Boolean, truncate: Boolean = true): String = { val sb = new StringBuilder val numCols = schema.fieldNames.length diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala index 1df38f7ff59cd..b5ac530444b79 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.types._ object TypedAggregateExpression { - def apply[A, B : Encoder, C : Encoder]( + def apply[A, B: Encoder, C: Encoder]( aggregator: Aggregator[A, B, C]): TypedAggregateExpression = { new TypedAggregateExpression( aggregator.asInstanceOf[Aggregator[Any, Any, Any]], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala index d45d2db62f3a9..d5e0d80076cbe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala @@ -256,7 +256,7 @@ private[spark] class SqlNewHadoopRDD[V: ClassTag]( val infos = c.newGetLocationInfo.invoke(split).asInstanceOf[Array[AnyRef]] Some(HadoopRDD.convertSplitLocationInfo(infos)) } catch { - case e : Exception => + case e: Exception => logDebug("Failed to use InputSplit#getLocationInfo.", e) None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala index fb97a03df60f4..c4b125e9d5f00 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala @@ -557,7 +557,7 @@ private[parquet] object CatalystSchemaConverter { } } - private def computeMinBytesForPrecision(precision : Int) : Int = { + private def computeMinBytesForPrecision(precision: Int): Int = { var numBytes = 1 while (math.pow(2.0, 8 * numBytes - 1) < math.pow(10.0, precision)) { numBytes += 1 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 index 93d32e1fb93ae..a567457dba3c5 100644 --- 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 @@ -34,7 +34,7 @@ import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter * materialize the right RDD (in case of the right RDD is nondeterministic). */ private[spark] -class UnsafeCartesianRDD(left : RDD[UnsafeRow], right : RDD[UnsafeRow], numFieldsOfRight: Int) +class UnsafeCartesianRDD(left: RDD[UnsafeRow], right: RDD[UnsafeRow], numFieldsOfRight: Int) extends CartesianRDD[UnsafeRow, UnsafeRow](left.sparkContext, left, right) { override def compute(split: Partition, context: TaskContext): Iterator[(UnsafeRow, UnsafeRow)] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 52735c9d7f8c4..8c68d9ee0a1ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -64,7 +64,7 @@ private[sql] trait SQLMetricValue[T] extends Serializable { /** * A wrapper of Long to avoid boxing and unboxing when using Accumulator */ -private[sql] class LongSQLMetricValue(private var _value : Long) extends SQLMetricValue[Long] { +private[sql] class LongSQLMetricValue(private var _value: Long) extends SQLMetricValue[Long] { def add(incr: Long): LongSQLMetricValue = { _value += incr diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala index a191759813de1..a4cb54e2bf2a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala @@ -94,7 +94,7 @@ private[sql] object FrequentItems extends Logging { (name, originalSchema.fields(index).dataType) }.toArray - val freqItems = df.select(cols.map(Column(_)) : _*).rdd.aggregate(countMaps)( + val freqItems = df.select(cols.map(Column(_)): _*).rdd.aggregate(countMaps)( seqOp = (counts, row) => { var i = 0 while (i < numCols) { @@ -115,7 +115,7 @@ private[sql] object FrequentItems extends Logging { } ) val justItems = freqItems.map(m => m.baseMap.keys.toArray) - val resultRow = Row(justItems : _*) + val resultRow = Row(justItems: _*) // append frequent Items to the column name for easy debugging val outputCols = colInfo.map { v => StructField(v._1 + "_freqItems", ArrayType(v._2, false)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala index e9b60841fc28c..05a9f377b9897 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala @@ -44,7 +44,7 @@ object Window { */ @scala.annotation.varargs def partitionBy(colName: String, colNames: String*): WindowSpec = { - spec.partitionBy(colName, colNames : _*) + spec.partitionBy(colName, colNames: _*) } /** @@ -53,7 +53,7 @@ object Window { */ @scala.annotation.varargs def partitionBy(cols: Column*): WindowSpec = { - spec.partitionBy(cols : _*) + spec.partitionBy(cols: _*) } /** @@ -62,7 +62,7 @@ object Window { */ @scala.annotation.varargs def orderBy(colName: String, colNames: String*): WindowSpec = { - spec.orderBy(colName, colNames : _*) + spec.orderBy(colName, colNames: _*) } /** @@ -71,7 +71,7 @@ object Window { */ @scala.annotation.varargs def orderBy(cols: Column*): WindowSpec = { - spec.orderBy(cols : _*) + spec.orderBy(cols: _*) } private def spec: WindowSpec = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 592d79df3109a..1ac62883a68ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -306,7 +306,7 @@ object functions extends LegacyFunctions { */ @scala.annotation.varargs def countDistinct(columnName: String, columnNames: String*): Column = - countDistinct(Column(columnName), columnNames.map(Column.apply) : _*) + countDistinct(Column(columnName), columnNames.map(Column.apply): _*) /** * Aggregate function: returns the first value in a group. @@ -768,7 +768,7 @@ object functions extends LegacyFunctions { */ @scala.annotation.varargs def array(colName: String, colNames: String*): Column = { - array((colName +: colNames).map(col) : _*) + array((colName +: colNames).map(col): _*) } /** @@ -977,7 +977,7 @@ object functions extends LegacyFunctions { */ @scala.annotation.varargs def struct(colName: String, colNames: String*): Column = { - struct((colName +: colNames).map(col) : _*) + struct((colName +: colNames).map(col): _*) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala index 467d8d62d1b7f..d2c31d6e04107 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala @@ -30,7 +30,7 @@ private class AggregatedDialect(dialects: List[JdbcDialect]) extends JdbcDialect require(dialects.nonEmpty) - override def canHandle(url : String): Boolean = + override def canHandle(url: String): Boolean = dialects.map(_.canHandle(url)).reduce(_ && _) override def getCatalystType( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index ca2d909e2cccc..8d58321d4887d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.types._ * send a null value to the database. */ @DeveloperApi -case class JdbcType(databaseTypeDefinition : String, jdbcNullType : Int) +case class JdbcType(databaseTypeDefinition: String, jdbcNullType: Int) /** * :: DeveloperApi :: @@ -60,7 +60,7 @@ abstract class JdbcDialect extends Serializable { * @return True if the dialect can be applied on the given jdbc url. * @throws NullPointerException if the url is null. */ - def canHandle(url : String): Boolean + def canHandle(url: String): Boolean /** * Get the custom datatype mapping for the given jdbc meta information. @@ -130,7 +130,7 @@ object JdbcDialects { * * @param dialect The new dialect. */ - def registerDialect(dialect: JdbcDialect) : Unit = { + def registerDialect(dialect: JdbcDialect): Unit = { dialects = dialect :: dialects.filterNot(_ == dialect) } @@ -139,7 +139,7 @@ object JdbcDialects { * * @param dialect The jdbc dialect. */ - def unregisterDialect(dialect : JdbcDialect) : Unit = { + def unregisterDialect(dialect: JdbcDialect): Unit = { dialects = dialects.filterNot(_ == dialect) } @@ -169,5 +169,5 @@ object JdbcDialects { * NOOP dialect object, always returning the neutral element. */ private object NoopDialect extends JdbcDialect { - override def canHandle(url : String): Boolean = true + override def canHandle(url: String): Boolean = true } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala index e1717049f383d..faae54e605c68 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala @@ -23,10 +23,13 @@ import org.apache.spark.sql.types.{BooleanType, DataType, LongType, MetadataBuil private case object MySQLDialect extends JdbcDialect { - override def canHandle(url : String): Boolean = url.startsWith("jdbc:mysql") + override def canHandle(url: String): Boolean = url.startsWith("jdbc:mysql") override def getCatalystType( - sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = { + sqlType: Int, + typeName: String, + size: Int, + md: MetadataBuilder): Option[DataType] = { if (sqlType == Types.VARBINARY && typeName.equals("BIT") && size != 1) { // This could instead be a BinaryType if we'd rather return bit-vectors of up to 64 bits as // byte arrays instead of longs. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala index 3258f3782d8cc..f952fc07fd387 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext /** An `Aggregator` that adds up any numeric type returned by the given function. */ -class SumOf[I, N : Numeric](f: I => N) extends Aggregator[I, N, N] { +class SumOf[I, N: Numeric](f: I => N) extends Aggregator[I, N, N] { val numeric = implicitly[Numeric[N]] override def zero: N = numeric.zero @@ -113,7 +113,7 @@ class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { import testImplicits._ - def sum[I, N : Numeric : Encoder](f: I => N): TypedColumn[I, N] = + def sum[I, N: Numeric: Encoder](f: I => N): TypedColumn[I, N] = new SumOf(f).toColumn test("typed aggregation: TypedAggregator") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala index 3a283a4e1f610..848f1af65508b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala @@ -27,7 +27,7 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("persist and unpersist") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS().select(expr("_2 + 1").as[Int]) + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS().select(expr("_2 + 1").as[Int]) val cached = ds.cache() // count triggers the caching action. It should not throw. cached.count() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 53b5f45c2d4a6..a3ed2e06165ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -30,7 +30,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("toDS") { - val data = Seq(("a", 1) , ("b", 2), ("c", 3)) + val data = Seq(("a", 1), ("b", 2), ("c", 3)) checkAnswer( data.toDS(), data: _*) @@ -87,7 +87,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("as case class / collect") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDF("a", "b").as[ClassData] + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDF("a", "b").as[ClassData] checkAnswer( ds, ClassData("a", 1), ClassData("b", 2), ClassData("c", 3)) @@ -105,7 +105,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("map") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.map(v => (v._1, v._2 + 1)), ("a", 2), ("b", 3), ("c", 4)) @@ -124,23 +124,23 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select(expr("_2 + 1").as[Int]), 2, 3, 4) } test("select 2") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select( expr("_1").as[String], - expr("_2").as[Int]) : Dataset[(String, Int)], + expr("_2").as[Int]): Dataset[(String, Int)], ("a", 1), ("b", 2), ("c", 3)) } test("select 2, primitive and tuple") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select( expr("_1").as[String], @@ -149,7 +149,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select 2, primitive and class") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select( expr("_1").as[String], @@ -158,7 +158,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select 2, primitive and class, fields reordered") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkDecoding( ds.select( expr("_1").as[String], @@ -167,28 +167,28 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("filter") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.filter(_._1 == "b"), ("b", 2)) } test("foreach") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() val acc = sparkContext.accumulator(0) ds.foreach(v => acc += v._2) assert(acc.value == 6) } test("foreachPartition") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() val acc = sparkContext.accumulator(0) ds.foreachPartition(_.foreach(v => acc += v._2)) assert(acc.value == 6) } test("reduce") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() assert(ds.reduce((a, b) => ("sum", a._2 + b._2)) == ("sum", 6)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 4ab148065a476..860e07c68cef1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -206,7 +206,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { StructType( StructField("f1", IntegerType, true) :: StructField("f2", IntegerType, true) :: Nil), - StructType(StructField("f1", LongType, true) :: Nil) , + StructType(StructField("f1", LongType, true) :: Nil), StructType( StructField("f1", LongType, true) :: StructField("f2", IntegerType, true) :: Nil)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index ab48e971b507a..f2e0a868f4b1a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -72,7 +72,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { /** * Writes `data` to a Parquet file, reads it back and check file contents. */ - protected def checkParquetFile[T <: Product : ClassTag: TypeTag](data: Seq[T]): Unit = { + protected def checkParquetFile[T <: Product: ClassTag: TypeTag](data: Seq[T]): Unit = { withParquetDataFrame(data)(r => checkAnswer(r, data.map(Row.fromTuple))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 1fa22e2933318..984e3fbc05e48 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -46,7 +46,7 @@ class JDBCSuite extends SparkFunSuite val testBytes = Array[Byte](99.toByte, 134.toByte, 135.toByte, 200.toByte, 205.toByte) val testH2Dialect = new JdbcDialect { - override def canHandle(url: String) : Boolean = url.startsWith("jdbc:h2") + override def canHandle(url: String): Boolean = url.startsWith("jdbc:h2") override def getCatalystType( sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = Some(StringType) @@ -489,7 +489,7 @@ class JDBCSuite extends SparkFunSuite test("Aggregated dialects") { val agg = new AggregatedDialect(List(new JdbcDialect { - override def canHandle(url: String) : Boolean = url.startsWith("jdbc:h2:") + override def canHandle(url: String): Boolean = url.startsWith("jdbc:h2:") override def getCatalystType( sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = if (sqlType % 2 == 0) { 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 index 599294dfbb7d7..d1d8a68f6d196 100644 --- 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive.thriftserver private[hive] object ReflectionUtils { - def setSuperField(obj : Object, fieldName: String, fieldValue: Object) { + def setSuperField(obj: Object, fieldName: String, fieldValue: Object) { setAncestorField(obj, 1, fieldName, fieldValue) } 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 03bc830df2034..9f9efe33e12a3 100644 --- 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 @@ -325,7 +325,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (ret != 0) { // For analysis exception, only the error is printed out to the console. rc.getException() match { - case e : AnalysisException => + case e: AnalysisException => err.println(s"""Error in query: ${e.getMessage}""") case _ => err.println(rc.getErrorMessage()) } @@ -369,7 +369,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (counter != 0) { responseMsg += s", Fetched $counter row(s)" } - console.printInfo(responseMsg , null) + console.printInfo(responseMsg, null) // Destroy the driver to release all the locks. driver.destroy() } else { 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 7a260e72eb459..c9df3c4a82c88 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 @@ -657,8 +657,8 @@ private[hive] trait HiveInspectors { case DecimalType() => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector case StructType(fields) => ObjectInspectorFactory.getStandardStructObjectInspector( - java.util.Arrays.asList(fields.map(f => f.name) : _*), - java.util.Arrays.asList(fields.map(f => toInspector(f.dataType)) : _*)) + java.util.Arrays.asList(fields.map(f => f.name): _*), + java.util.Arrays.asList(fields.map(f => toInspector(f.dataType)): _*)) } /** @@ -905,8 +905,8 @@ private[hive] trait HiveInspectors { getListTypeInfo(elemType.toTypeInfo) case StructType(fields) => getStructTypeInfo( - java.util.Arrays.asList(fields.map(_.name) : _*), - java.util.Arrays.asList(fields.map(_.dataType.toTypeInfo) : _*)) + java.util.Arrays.asList(fields.map(_.name): _*), + java.util.Arrays.asList(fields.map(_.dataType.toTypeInfo): _*)) case MapType(keyType, valueType, _) => getMapTypeInfo(keyType.toTypeInfo, valueType.toTypeInfo) case BinaryType => binaryTypeInfo 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 56cab1aee89df..912cd41173a2a 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 @@ -181,7 +181,7 @@ private[hive] case class HiveSimpleUDF( val ret = FunctionRegistry.invoke( method, function, - conversionHelper.convertIfNecessary(inputs : _*): _*) + conversionHelper.convertIfNecessary(inputs: _*): _*) unwrap(ret, returnInspector) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala index 3b867bbfa1817..ad28345a667d0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala @@ -118,8 +118,8 @@ class HiveInspectorSuite extends SparkFunSuite with HiveInspectors { case DecimalType() => PrimitiveObjectInspectorFactory.writableHiveDecimalObjectInspector case StructType(fields) => ObjectInspectorFactory.getStandardStructObjectInspector( - java.util.Arrays.asList(fields.map(f => f.name) : _*), - java.util.Arrays.asList(fields.map(f => toWritableInspector(f.dataType)) : _*)) + java.util.Arrays.asList(fields.map(f => f.name): _*), + java.util.Arrays.asList(fields.map(f => toWritableInspector(f.dataType)): _*)) } def checkDataType(dt1: Seq[DataType], dt2: Seq[DataType]): Unit = { 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 da7303c791064..40e9c9362cf5e 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 @@ -154,8 +154,8 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef } 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=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)) From c48f2a3a5fd714ad2ff19b29337e55583988431e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 12 Jan 2016 11:50:33 +0000 Subject: [PATCH 076/110] [SPARK-7615][MLLIB] MLLIB Word2Vec wordVectors divided by Euclidean Norm equals to zero Cosine similarity with 0 vector should be 0 Related to https://github.com/apache/spark/pull/10152 Author: Sean Owen Closes #10696 from srowen/SPARK-7615. --- .../scala/org/apache/spark/mllib/feature/Word2Vec.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) 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 dc5d070890d5d..dee898827f30f 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 @@ -543,7 +543,12 @@ class Word2VecModel private[spark] ( val cosVec = cosineVec.map(_.toDouble) var ind = 0 while (ind < numWords) { - cosVec(ind) /= wordVecNorms(ind) + val norm = wordVecNorms(ind) + if (norm == 0.0) { + cosVec(ind) = 0.0 + } else { + cosVec(ind) /= norm + } ind += 1 } wordList.zip(cosVec) From 9c7f34af37ef328149c1d66b4689d80a1589e1cc Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 12 Jan 2016 12:13:32 +0000 Subject: [PATCH 077/110] [SPARK-5273][MLLIB][DOCS] Improve documentation examples for LinearRegression Use a much smaller step size in LinearRegressionWithSGD MLlib examples to achieve a reasonable RMSE. Our training folks hit this exact same issue when concocting an example and had the same solution. Author: Sean Owen Closes #10675 from srowen/SPARK-5273. --- docs/mllib-linear-methods.md | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 20b35612cab95..aac8f7560a4f8 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -590,7 +590,8 @@ val parsedData = data.map { line => // Building the model val numIterations = 100 -val model = LinearRegressionWithSGD.train(parsedData, numIterations) +val stepSize = 0.00000001 +val model = LinearRegressionWithSGD.train(parsedData, numIterations, stepSize) // Evaluate model on training examples and compute training error val valuesAndPreds = parsedData.map { point => @@ -655,8 +656,9 @@ public class LinearRegression { // Building the model int numIterations = 100; + double stepSize = 0.00000001; final LinearRegressionModel model = - LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), numIterations); + LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), numIterations, stepSize); // Evaluate model on training examples and compute training error JavaRDD> valuesAndPreds = parsedData.map( @@ -706,7 +708,7 @@ data = sc.textFile("data/mllib/ridge-data/lpsa.data") parsedData = data.map(parsePoint) # Build the model -model = LinearRegressionWithSGD.train(parsedData) +model = LinearRegressionWithSGD.train(parsedData, iterations=100, step=0.00000001) # Evaluate the model on training data valuesAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) From 9f0995bb0d0bbe5d9b15a1ca9fa18e246ff90d66 Mon Sep 17 00:00:00 2001 From: Tommy YU Date: Tue, 12 Jan 2016 13:20:04 +0000 Subject: [PATCH 078/110] [SPARK-12638][API DOC] Parameter explanation not very accurate for rdd function "aggregate" Currently, RDD function aggregate's parameter doesn't explain well, especially parameter "zeroValue". It's helpful to let junior scala user know that "zeroValue" attend both "seqOp" and "combOp" phase. Author: Tommy YU Closes #10587 from Wenpei/rdd_aggregate_doc. --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 14 ++++++++++++++ 1 file changed, 14 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 de7102f5b6245..53e01a0dbfc06 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -970,6 +970,13 @@ abstract class RDD[T: ClassTag]( * apply the fold to each element sequentially in some defined ordering. For functions * that are not commutative, the result may differ from that of a fold applied to a * non-distributed collection. + * + * @param zeroValue the initial value for the accumulated result of each partition for the `op` + * operator, and also the initial value for the combine results from different + * partitions for the `op` operator - this will typically be the neutral + * element (e.g. `Nil` for list concatenation or `0` for summation) + * @param op an operator used to both accumulate results within a partition and combine results + * from different partitions */ def fold(zeroValue: T)(op: (T, T) => T): T = withScope { // Clone the zero value since we will also be serializing it as part of tasks @@ -988,6 +995,13 @@ abstract class RDD[T: ClassTag]( * and one operation for merging two U's, as in scala.TraversableOnce. Both of these functions are * allowed to modify and return their first argument instead of creating a new U to avoid memory * allocation. + * + * @param zeroValue the initial value for the accumulated result of each partition for the + * `seqOp` operator, and also the initial value for the combine results from + * different partitions for the `combOp` operator - this will typically be the + * neutral element (e.g. `Nil` for list concatenation or `0` for summation) + * @param seqOp an operator used to accumulate results within a partition + * @param combOp an associative operator used to combine results from different partitions */ def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = withScope { // Clone the zero value since we will also be serializing it as part of tasks From 7e15044d9d9f9839c8d422bae71f27e855d559b4 Mon Sep 17 00:00:00 2001 From: Yucai Yu Date: Tue, 12 Jan 2016 13:23:23 +0000 Subject: [PATCH 079/110] [SPARK-12582][TEST] IndexShuffleBlockResolverSuite fails in windows [SPARK-12582][Test] IndexShuffleBlockResolverSuite fails in windows * IndexShuffleBlockResolverSuite fails in windows due to file is not closed. * mv IndexShuffleBlockResolverSuite.scala from "test/java" to "test/scala". https://issues.apache.org/jira/browse/SPARK-12582 Author: Yucai Yu Closes #10526 from yucai/master. --- .../sort/IndexShuffleBlockResolverSuite.scala | 51 ++++++++++++------- 1 file changed, 34 insertions(+), 17 deletions(-) rename core/src/test/{java => scala}/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala (87%) diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala similarity index 87% rename from core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala rename to core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala index f200ff36c7dd5..d21ce73f4021e 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala @@ -19,18 +19,18 @@ package org.apache.spark.shuffle.sort import java.io.{File, FileInputStream, FileOutputStream} +import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.Matchers._ import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer -import org.mockito.{Mock, MockitoAnnotations} import org.scalatest.BeforeAndAfterEach +import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.shuffle.IndexShuffleBlockResolver import org.apache.spark.storage._ import org.apache.spark.util.Utils -import org.apache.spark.{SparkConf, SparkFunSuite} class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEach { @@ -64,12 +64,15 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } test("commit shuffle files multiple times") { - val lengths = Array[Long](10, 0, 20) val resolver = new IndexShuffleBlockResolver(conf, blockManager) + val lengths = Array[Long](10, 0, 20) val dataTmp = File.createTempFile("shuffle", null, tempDir) val out = new FileOutputStream(dataTmp) - out.write(new Array[Byte](30)) - out.close() + Utils.tryWithSafeFinally { + out.write(new Array[Byte](30)) + } { + out.close() + } resolver.writeIndexFileAndCommit(1, 2, lengths, dataTmp) val dataFile = resolver.getDataFile(1, 2) @@ -77,12 +80,15 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa assert(dataFile.length() === 30) assert(!dataTmp.exists()) + val lengths2 = new Array[Long](3) val dataTmp2 = File.createTempFile("shuffle", null, tempDir) val out2 = new FileOutputStream(dataTmp2) - val lengths2 = new Array[Long](3) - out2.write(Array[Byte](1)) - out2.write(new Array[Byte](29)) - out2.close() + Utils.tryWithSafeFinally { + out2.write(Array[Byte](1)) + out2.write(new Array[Byte](29)) + } { + out2.close() + } resolver.writeIndexFileAndCommit(1, 2, lengths2, dataTmp2) assert(lengths2.toSeq === lengths.toSeq) assert(dataFile.exists()) @@ -90,20 +96,27 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa assert(!dataTmp2.exists()) // The dataFile should be the previous one - val in = new FileInputStream(dataFile) val firstByte = new Array[Byte](1) - in.read(firstByte) + val in = new FileInputStream(dataFile) + Utils.tryWithSafeFinally { + in.read(firstByte) + } { + in.close() + } assert(firstByte(0) === 0) // remove data file dataFile.delete() + val lengths3 = Array[Long](10, 10, 15) val dataTmp3 = File.createTempFile("shuffle", null, tempDir) val out3 = new FileOutputStream(dataTmp3) - val lengths3 = Array[Long](10, 10, 15) - out3.write(Array[Byte](2)) - out3.write(new Array[Byte](34)) - out3.close() + Utils.tryWithSafeFinally { + out3.write(Array[Byte](2)) + out3.write(new Array[Byte](34)) + } { + out3.close() + } resolver.writeIndexFileAndCommit(1, 2, lengths3, dataTmp3) assert(lengths3.toSeq != lengths.toSeq) assert(dataFile.exists()) @@ -111,9 +124,13 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa assert(!dataTmp2.exists()) // The dataFile should be the previous one - val in2 = new FileInputStream(dataFile) val firstByte2 = new Array[Byte](1) - in2.read(firstByte2) + val in2 = new FileInputStream(dataFile) + Utils.tryWithSafeFinally { + in2.read(firstByte2) + } { + in2.close() + } assert(firstByte2(0) === 2) } } From 1d8887953018b2e12b6ee47a76e50e542c836b80 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 12 Jan 2016 10:58:57 -0800 Subject: [PATCH 080/110] [SPARK-12762][SQL] Add unit test for SimplifyConditionals optimization rule This pull request does a few small things: 1. Separated if simplification from BooleanSimplification and created a new rule SimplifyConditionals. In the future we can also simplify other conditional expressions here. 2. Added unit test for SimplifyConditionals. 3. Renamed SimplifyCaseConversionExpressionsSuite to SimplifyStringCaseConversionSuite Author: Reynold Xin Closes #10716 from rxin/SPARK-12762. --- .../expressions/conditionalExpressions.scala | 10 ++-- .../sql/catalyst/optimizer/Optimizer.scala | 10 ++++ .../optimizer/CombiningLimitsSuite.scala | 3 +- .../optimizer/SimplifyConditionalSuite.scala | 50 +++++++++++++++++++ ...> SimplifyStringCaseConversionSuite.scala} | 3 +- 5 files changed, 69 insertions(+), 7 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala rename sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/{SimplifyCaseConversionExpressionsSuite.scala => SimplifyStringCaseConversionSuite.scala} (96%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 19da849d2bec9..379e62a26eb47 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -45,7 +45,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi override def dataType: DataType = trueValue.dataType override def eval(input: InternalRow): Any = { - if (true == predicate.eval(input)) { + if (java.lang.Boolean.TRUE.equals(predicate.eval(input))) { trueValue.eval(input) } else { falseValue.eval(input) @@ -141,8 +141,8 @@ case class CaseWhen(branches: Seq[Expression]) extends CaseWhenLike { } } - /** Written in imperative fashion for performance considerations. */ override def eval(input: InternalRow): Any = { + // Written in imperative fashion for performance considerations val len = branchesArr.length var i = 0 // If all branches fail and an elseVal is not provided, the whole statement @@ -389,7 +389,7 @@ case class Least(children: Seq[Expression]) extends Expression { val evalChildren = children.map(_.gen(ctx)) val first = evalChildren(0) val rest = evalChildren.drop(1) - def updateEval(eval: GeneratedExpressionCode): String = + def updateEval(eval: GeneratedExpressionCode): String = { s""" ${eval.code} if (!${eval.isNull} && (${ev.isNull} || @@ -398,6 +398,7 @@ case class Least(children: Seq[Expression]) extends Expression { ${ev.value} = ${eval.value}; } """ + } s""" ${first.code} boolean ${ev.isNull} = ${first.isNull}; @@ -447,7 +448,7 @@ case class Greatest(children: Seq[Expression]) extends Expression { val evalChildren = children.map(_.gen(ctx)) val first = evalChildren(0) val rest = evalChildren.drop(1) - def updateEval(eval: GeneratedExpressionCode): String = + def updateEval(eval: GeneratedExpressionCode): String = { s""" ${eval.code} if (!${eval.isNull} && (${ev.isNull} || @@ -456,6 +457,7 @@ case class Greatest(children: Seq[Expression]) extends Expression { ${ev.value} = ${eval.value}; } """ + } s""" ${first.code} boolean ${ev.isNull} = ${first.isNull}; 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 b70bc184d0a5e..487431f8925a3 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 @@ -63,6 +63,7 @@ abstract class Optimizer extends RuleExecutor[LogicalPlan] { ConstantFolding, LikeSimplification, BooleanSimplification, + SimplifyConditionals, RemoveDispensableExpressions, SimplifyFilters, SimplifyCasts, @@ -608,7 +609,16 @@ object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { case Not(a And b) => Or(Not(a), Not(b)) case Not(Not(e)) => e + } + } +} +/** + * Simplifies conditional expressions (if / case). + */ +object SimplifyConditionals extends Rule[LogicalPlan] with PredicateHelper { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case q: LogicalPlan => q transformExpressionsUp { case If(TrueLiteral, trueValue, _) => trueValue case If(FalseLiteral, _, falseValue) => falseValue } 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 9fe2b2d1f48ca..87ad81db11b64 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 @@ -34,7 +34,8 @@ class CombiningLimitsSuite extends PlanTest { Batch("Constant Folding", FixedPoint(10), NullPropagation, ConstantFolding, - BooleanSimplification) :: Nil + BooleanSimplification, + SimplifyConditionals) :: Nil } val testRelation = LocalRelation('a.int, 'b.int, 'c.int) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala new file mode 100644 index 0000000000000..8e5d7ef3c9d49 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.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.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ + + +class SimplifyConditionalSuite extends PlanTest with PredicateHelper { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("SimplifyConditionals", FixedPoint(50), SimplifyConditionals) :: Nil + } + + protected def assertEquivalent(e1: Expression, e2: Expression): Unit = { + val correctAnswer = Project(Alias(e2, "out")() :: Nil, OneRowRelation).analyze + val actual = Optimize.execute(Project(Alias(e1, "out")() :: Nil, OneRowRelation).analyze) + comparePlans(actual, correctAnswer) + } + + test("simplify if") { + assertEquivalent( + If(TrueLiteral, Literal(10), Literal(20)), + Literal(10)) + + assertEquivalent( + If(FalseLiteral, Literal(10), Literal(20)), + Literal(20)) + } + +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyStringCaseConversionSuite.scala similarity index 96% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyStringCaseConversionSuite.scala index 41455221cfdc6..24413e7a2a3f0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyStringCaseConversionSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.optimizer -/* Implicit conversions */ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ @@ -25,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.rules._ -class SimplifyCaseConversionExpressionsSuite extends PlanTest { +class SimplifyStringCaseConversionSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = From 508592b1bae3b2c88350ddfc1d909892f236ce5f Mon Sep 17 00:00:00 2001 From: Robert Kruszewski Date: Tue, 12 Jan 2016 11:09:28 -0800 Subject: [PATCH 081/110] [SPARK-9843][SQL] Make catalyst optimizer pass pluggable at runtime Let me know whether you'd like to see it in other place Author: Robert Kruszewski Closes #10210 from robert3005/feature/pluggable-optimizer. --- .../spark/sql/ExperimentalMethods.scala | 5 ++++ .../org/apache/spark/sql/SQLContext.scala | 4 +-- .../spark/sql/execution/SparkOptimizer.scala | 27 +++++++++++++++++++ .../apache/spark/sql/SQLContextSuite.scala | 12 +++++++++ 4 files changed, 46 insertions(+), 2 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala index 717709e4f9312..deed45d273c33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule /** * :: Experimental :: @@ -42,4 +44,7 @@ class ExperimentalMethods protected[sql](sqlContext: SQLContext) { @Experimental var extraStrategies: Seq[Strategy] = Nil + @Experimental + var extraOptimizations: Seq[Rule[LogicalPlan]] = Nil + } 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 61c74f83409e9..6721d9c40748b 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 @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.errors.DialectException import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer} +import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Range} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ @@ -202,7 +202,7 @@ class SQLContext private[sql]( } @transient - protected[sql] lazy val optimizer: Optimizer = DefaultOptimizer + protected[sql] lazy val optimizer: Optimizer = new SparkOptimizer(this) @transient protected[sql] val ddlParser = new DDLParser(sqlParser.parse(_)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala new file mode 100644 index 0000000000000..edaf3b36aa52e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.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.execution + +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.optimizer._ + +class SparkOptimizer(val sqlContext: SQLContext) + extends Optimizer { + override def batches: Seq[Batch] = super.batches :+ Batch( + "User Provided Optimizers", FixedPoint(100), sqlContext.experimental.extraOptimizations: _*) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index 1994dacfc4dfa..14b9448d260f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -18,9 +18,15 @@ package org.apache.spark.sql import org.apache.spark.{SharedSparkContext, SparkFunSuite} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule class SQLContextSuite extends SparkFunSuite with SharedSparkContext{ + object DummyRule extends Rule[LogicalPlan] { + def apply(p: LogicalPlan): LogicalPlan = p + } + test("getOrCreate instantiates SQLContext") { val sqlContext = SQLContext.getOrCreate(sc) assert(sqlContext != null, "SQLContext.getOrCreate returned null") @@ -65,4 +71,10 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext{ session2.sql("select myadd(1, 2)").explain() } } + + test("Catalyst optimization passes are modifiable at runtime") { + val sqlContext = SQLContext.getOrCreate(sc) + sqlContext.experimental.extraOptimizations = Seq(DummyRule) + assert(sqlContext.optimizer.batches.flatMap(_.rules).contains(DummyRule)) + } } From 0ed430e315b9a409490a3604a619321b476cb520 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 12 Jan 2016 11:13:08 -0800 Subject: [PATCH 082/110] [SPARK-12768][SQL] Remove CaseKeyWhen expression This patch removes CaseKeyWhen expression and replaces it with a factory method that generates the equivalent CaseWhen. This reduces the amount of code we'd need to maintain in the future for both code generation and optimizer. Note that we introduced CaseKeyWhen to avoid duplicate evaluations of the key. This is no longer a problem because we now have common subexpression elimination. Author: Reynold Xin Closes #10722 from rxin/SPARK-12768. --- .../catalyst/analysis/HiveTypeCoercion.scala | 20 +- .../expressions/conditionalExpressions.scala | 187 ++++-------------- .../analysis/HiveTypeCoercionSuite.scala | 2 +- 3 files changed, 38 insertions(+), 171 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 e326ea782700c..75c36d93108df 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 @@ -638,8 +638,7 @@ object HiveTypeCoercion { */ object CaseWhenCoercion extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions { - case c: CaseWhenLike if c.childrenResolved && !c.valueTypesEqual => - logDebug(s"Input values for null casting ${c.valueTypes.mkString(",")}") + case c: CaseWhen if c.childrenResolved && !c.valueTypesEqual => val maybeCommonType = findWiderCommonType(c.valueTypes) maybeCommonType.map { commonType => val castedBranches = c.branches.grouped(2).map { @@ -649,22 +648,7 @@ object HiveTypeCoercion { Seq(Cast(elseVal, commonType)) case other => other }.reduce(_ ++ _) - c match { - case _: CaseWhen => CaseWhen(castedBranches) - case CaseKeyWhen(key, _) => CaseKeyWhen(key, castedBranches) - } - }.getOrElse(c) - - case c: CaseKeyWhen if c.childrenResolved && !c.resolved => - val maybeCommonType = - findWiderCommonType((c.key +: c.whenList).map(_.dataType)) - maybeCommonType.map { commonType => - val castedBranches = c.branches.grouped(2).map { - case Seq(whenExpr, thenExpr) if whenExpr.dataType != commonType => - Seq(Cast(whenExpr, commonType), thenExpr) - case other => other - }.reduce(_ ++ _) - CaseKeyWhen(Cast(c.key, commonType), castedBranches) + CaseWhen(castedBranches) }.getOrElse(c) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 379e62a26eb47..5a1462433d583 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.util.{sequenceOption, TypeUtils} +import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @@ -78,17 +78,23 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi override def sql: String = s"(IF(${predicate.sql}, ${trueValue.sql}, ${falseValue.sql}))" } -trait CaseWhenLike extends Expression { +/** + * Case statements of the form "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END". + * When a = true, returns b; when c = true, returns d; else returns e. + */ +case class CaseWhen(branches: Seq[Expression]) extends Expression { + + // Use private[this] Array to speed up evaluation. + @transient private[this] lazy val branchesArr = branches.toArray - // Note that `branches` are considered in consecutive pairs (cond, val), and the optional last - // element is the value for the default catch-all case (if provided). - // Hence, `branches` consists of at least two elements, and can have an odd or even length. - def branches: Seq[Expression] + override def children: Seq[Expression] = branches @transient lazy val whenList = branches.sliding(2, 2).collect { case Seq(whenExpr, _) => whenExpr }.toSeq + @transient lazy val thenList = branches.sliding(2, 2).collect { case Seq(_, thenExpr) => thenExpr }.toSeq + val elseValue = if (branches.length % 2 == 0) None else Option(branches.last) // both then and else expressions should be considered. @@ -97,47 +103,26 @@ trait CaseWhenLike extends Expression { case Seq(dt1, dt2) => dt1.sameType(dt2) } - override def checkInputDataTypes(): TypeCheckResult = { - if (valueTypesEqual) { - checkTypesInternal() - } else { - TypeCheckResult.TypeCheckFailure( - "THEN and ELSE expressions should all be same type or coercible to a common type") - } - } - - protected def checkTypesInternal(): TypeCheckResult - override def dataType: DataType = thenList.head.dataType override def nullable: Boolean = { // If no value is nullable and no elseValue is provided, the whole statement defaults to null. thenList.exists(_.nullable) || elseValue.map(_.nullable).getOrElse(true) } -} - -// scalastyle:off -/** - * Case statements of the form "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END". - * Refer to this link for the corresponding semantics: - * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions - */ -// scalastyle:on -case class CaseWhen(branches: Seq[Expression]) extends CaseWhenLike { - - // Use private[this] Array to speed up evaluation. - @transient private[this] lazy val branchesArr = branches.toArray - override def children: Seq[Expression] = branches - - override protected def checkTypesInternal(): TypeCheckResult = { - if (whenList.forall(_.dataType == BooleanType)) { - TypeCheckResult.TypeCheckSuccess + override def checkInputDataTypes(): TypeCheckResult = { + if (valueTypesEqual) { + if (whenList.forall(_.dataType == BooleanType)) { + TypeCheckResult.TypeCheckSuccess + } else { + val index = whenList.indexWhere(_.dataType != BooleanType) + TypeCheckResult.TypeCheckFailure( + s"WHEN expressions in CaseWhen should all be boolean type, " + + s"but the ${index + 1}th when expression's type is ${whenList(index)}") + } } else { - val index = whenList.indexWhere(_.dataType != BooleanType) TypeCheckResult.TypeCheckFailure( - s"WHEN expressions in CaseWhen should all be boolean type, " + - s"but the ${index + 1}th when expression's type is ${whenList(index)}") + "THEN and ELSE expressions should all be same type or coercible to a common type") } } @@ -227,125 +212,23 @@ case class CaseWhen(branches: Seq[Expression]) extends CaseWhenLike { } } -// scalastyle:off /** * Case statements of the form "CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END". - * Refer to this link for the corresponding semantics: - * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions + * When a = b, returns c; when a = d, returns e; else returns f. */ -// scalastyle:on -case class CaseKeyWhen(key: Expression, branches: Seq[Expression]) extends CaseWhenLike { - - // Use private[this] Array to speed up evaluation. - @transient private[this] lazy val branchesArr = branches.toArray - - override def children: Seq[Expression] = key +: branches - - override protected def checkTypesInternal(): TypeCheckResult = { - if ((key +: whenList).map(_.dataType).distinct.size > 1) { - TypeCheckResult.TypeCheckFailure( - "key and WHEN expressions should all be same type or coercible to a common type") - } else { - TypeCheckResult.TypeCheckSuccess - } - } - - private def evalElse(input: InternalRow): Any = { - if (branchesArr.length % 2 == 0) { - null - } else { - branchesArr(branchesArr.length - 1).eval(input) - } - } - - /** Written in imperative fashion for performance considerations. */ - override def eval(input: InternalRow): Any = { - val evaluatedKey = key.eval(input) - // If key is null, we can just return the else part or null if there is no else. - // If key is not null but doesn't match any when part, we need to return - // the else part or null if there is no else, according to Hive's semantics. - if (evaluatedKey != null) { - val len = branchesArr.length - var i = 0 - while (i < len - 1) { - if (evaluatedKey == branchesArr(i).eval(input)) { - return branchesArr(i + 1).eval(input) - } - i += 2 - } - } - evalElse(input) - } - - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val keyEval = key.gen(ctx) - val len = branchesArr.length - val got = ctx.freshName("got") - - val cases = (0 until len/2).map { i => - val cond = branchesArr(i * 2).gen(ctx) - val res = branchesArr(i * 2 + 1).gen(ctx) - s""" - if (!$got) { - ${cond.code} - if (!${cond.isNull} && ${ctx.genEqual(key.dataType, keyEval.value, cond.value)}) { - $got = true; - ${res.code} - ${ev.isNull} = ${res.isNull}; - ${ev.value} = ${res.value}; - } - } - """ - }.mkString("\n") - - val other = if (len % 2 == 1) { - val res = branchesArr(len - 1).gen(ctx) - s""" - if (!$got) { - ${res.code} - ${ev.isNull} = ${res.isNull}; - ${ev.value} = ${res.value}; - } - """ - } else { - "" - } - - s""" - boolean $got = false; - boolean ${ev.isNull} = true; - ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; - ${keyEval.code} - if (!${keyEval.isNull}) { - $cases +object CaseKeyWhen { + def apply(key: Expression, branches: Seq[Expression]): CaseWhen = { + val newBranches = branches.zipWithIndex.map { case (expr, i) => + if (i % 2 == 0 && i != branches.size - 1) { + // If this expression is at even position, then it is either a branch condition, or + // the very last value that is the "else value". The "i != branches.size - 1" makes + // sure we are not adding an EqualTo to the "else value". + EqualTo(key, expr) + } else { + expr } - $other - """ - } - - override def toString: String = { - s"CASE $key" + branches.sliding(2, 2).map { - case Seq(cond, value) => s" WHEN $cond THEN $value" - case Seq(elseValue) => s" ELSE $elseValue" - }.mkString - } - - override def sql: String = { - val keySQL = key.sql - val branchesSQL = branches.map(_.sql) - val (cases, maybeElse) = if (branches.length % 2 == 0) { - (branchesSQL, None) - } else { - (branchesSQL.init, Some(branchesSQL.last)) } - - val head = s"CASE $keySQL " - val tail = maybeElse.map(e => s" ELSE $e").getOrElse("") + " END" - val body = cases.grouped(2).map { - case Seq(whenExpr, thenExpr) => s"WHEN $whenExpr THEN $thenExpr" - }.mkString(" ") - - head + body + tail + CaseWhen(newBranches) } } 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 58d808c55860d..23b11af9ac087 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 @@ -299,7 +299,7 @@ class HiveTypeCoercionSuite extends PlanTest { } test("type coercion for CaseKeyWhen") { - ruleTest(HiveTypeCoercion.CaseWhenCoercion, + ruleTest(HiveTypeCoercion.ImplicitTypeCasts, CaseKeyWhen(Literal(1.toShort), Seq(Literal(1), Literal("a"))), CaseKeyWhen(Cast(Literal(1.toShort), IntegerType), Seq(Literal(1), Literal("a"))) ) From 0d543b98f3e3da5053f0476f4647a765460861f3 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 12 Jan 2016 12:56:52 -0800 Subject: [PATCH 083/110] Revert "[SPARK-12692][BUILD][SQL] Scala style: Fix the style violation (Space before "," or ":")" This reverts commit 8cfa218f4f1b05f4d076ec15dd0a033ad3e4500d. --- scalastyle-config.xml | 2 +- .../spark/sql/catalyst/ScalaReflection.scala | 6 ++-- .../apache/spark/sql/catalyst/SqlParser.scala | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 6 ++-- .../catalyst/analysis/FunctionRegistry.scala | 4 +-- .../catalyst/analysis/HiveTypeCoercion.scala | 2 +- .../spark/sql/catalyst/dsl/package.scala | 4 +-- .../catalyst/encoders/ExpressionEncoder.scala | 2 +- .../spark/sql/catalyst/encoders/package.scala | 2 +- .../sql/catalyst/expressions/Expression.scala | 2 +- .../expressions/stringExpressions.scala | 6 ++-- .../plans/logical/basicOperators.scala | 6 ++-- .../sql/catalyst/util/NumberConverter.scala | 2 +- .../apache/spark/sql/types/ArrayType.scala | 2 +- .../org/apache/spark/sql/types/Decimal.scala | 2 -- .../encoders/EncoderErrorMessageSuite.scala | 2 +- .../encoders/ExpressionEncoderSuite.scala | 6 ++-- .../BooleanSimplificationSuite.scala | 2 +- .../scala/org/apache/spark/sql/Column.scala | 4 +-- .../org/apache/spark/sql/DataFrame.scala | 36 +++++++++---------- .../apache/spark/sql/DataFrameHolder.scala | 2 +- .../spark/sql/DataFrameNaFunctions.scala | 8 ++--- .../apache/spark/sql/DataFrameReader.scala | 6 ++-- .../scala/org/apache/spark/sql/Dataset.scala | 18 +++++----- .../org/apache/spark/sql/GroupedData.scala | 10 +++--- .../org/apache/spark/sql/GroupedDataset.scala | 8 ++--- .../org/apache/spark/sql/SQLContext.scala | 12 +++---- .../org/apache/spark/sql/SQLImplicits.scala | 10 +++--- .../org/apache/spark/sql/api/r/SQLUtils.scala | 2 +- .../apache/spark/sql/execution/Exchange.scala | 2 +- .../spark/sql/execution/Queryable.scala | 2 +- .../aggregate/TypedAggregateExpression.scala | 2 +- .../datasources/SqlNewHadoopRDD.scala | 2 +- .../parquet/CatalystSchemaConverter.scala | 2 +- .../execution/joins/CartesianProduct.scala | 2 +- .../sql/execution/metric/SQLMetrics.scala | 2 +- .../sql/execution/stat/FrequentItems.scala | 4 +-- .../apache/spark/sql/expressions/Window.scala | 8 ++--- .../org/apache/spark/sql/functions.scala | 6 ++-- .../spark/sql/jdbc/AggregatedDialect.scala | 2 +- .../apache/spark/sql/jdbc/JdbcDialects.scala | 10 +++--- .../apache/spark/sql/jdbc/MySQLDialect.scala | 7 ++-- .../spark/sql/DatasetAggregatorSuite.scala | 4 +-- .../apache/spark/sql/DatasetCacheSuite.scala | 2 +- .../org/apache/spark/sql/DatasetSuite.scala | 26 +++++++------- .../datasources/json/JsonSuite.scala | 2 +- .../datasources/parquet/ParquetIOSuite.scala | 2 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +-- .../hive/thriftserver/ReflectionUtils.scala | 2 +- .../hive/thriftserver/SparkSQLCLIDriver.scala | 4 +-- .../spark/sql/hive/HiveInspectors.scala | 8 ++--- .../org/apache/spark/sql/hive/hiveUDFs.scala | 2 +- .../spark/sql/hive/HiveInspectorSuite.scala | 4 +-- .../sql/hive/InsertIntoHiveTableSuite.scala | 4 +-- 54 files changed, 141 insertions(+), 150 deletions(-) diff --git a/scalastyle-config.xml b/scalastyle-config.xml index b873b627219f2..2439a1f715aba 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -218,7 +218,7 @@ This file is divided into 3 sections: - + COLON, COMMA 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 23fea0e2832a1..79f723cf9b8a0 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 @@ -49,7 +49,7 @@ object ScalaReflection extends ScalaReflection { * Unlike `schemaFor`, this function doesn't do any massaging of types into the Spark SQL type * system. As a result, ObjectType will be returned for things like boxed Integers */ - def dataTypeFor[T: TypeTag]: DataType = dataTypeFor(localTypeOf[T]) + def dataTypeFor[T : TypeTag]: DataType = dataTypeFor(localTypeOf[T]) private def dataTypeFor(tpe: `Type`): DataType = ScalaReflectionLock.synchronized { tpe match { @@ -116,7 +116,7 @@ object ScalaReflection extends ScalaReflection { * from ordinal 0 (since there are no names to map to). The actual location can be moved by * calling resolve/bind with a new schema. */ - def constructorFor[T: TypeTag]: Expression = { + def constructorFor[T : TypeTag]: Expression = { val tpe = localTypeOf[T] val clsName = getClassNameFromType(tpe) val walkedTypePath = s"""- root class: "${clsName}"""" :: Nil @@ -386,7 +386,7 @@ object ScalaReflection extends ScalaReflection { * * the element type of [[Array]] or [[Seq]]: `array element class: "abc.xyz.MyClass"` * * the field of [[Product]]: `field (class: "abc.xyz.MyClass", name: "myField")` */ - def extractorsFor[T: TypeTag](inputObject: Expression): CreateNamedStruct = { + def extractorsFor[T : TypeTag](inputObject: Expression): CreateNamedStruct = { val tpe = localTypeOf[T] val clsName = getClassNameFromType(tpe) val walkedTypePath = s"""- root class: "${clsName}"""" :: Nil 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 6ec408a673c79..2a132d8b82bef 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 @@ -203,7 +203,7 @@ object SqlParser extends AbstractSparkSQLParser with DataTypeParser { ) protected lazy val ordering: Parser[Seq[SortOrder]] = - ( rep1sep(expression ~ direction.?, ",") ^^ { + ( rep1sep(expression ~ direction.? , ",") ^^ { case exps => exps.map(pair => SortOrder(pair._1, pair._2.getOrElse(Ascending))) } ) 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 d16880bc4a9c9..8a33af8207350 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 @@ -84,7 +84,7 @@ class Analyzer( ResolveAggregateFunctions :: DistinctAggregationRewriter(conf) :: HiveTypeCoercion.typeCoercionRules ++ - extendedResolutionRules: _*), + extendedResolutionRules : _*), Batch("Nondeterministic", Once, PullOutNondeterministic), Batch("UDF", Once, @@ -110,7 +110,7 @@ class Analyzer( // Taking into account the reasonableness and the implementation complexity, // here use the CTE definition first, check table name only and ignore database name // see https://github.com/apache/spark/pull/4929#discussion_r27186638 for more info - case u: UnresolvedRelation => + case u : UnresolvedRelation => val substituted = cteRelations.get(u.tableIdentifier.table).map { relation => val withAlias = u.alias.map(Subquery(_, relation)) withAlias.getOrElse(relation) @@ -889,7 +889,7 @@ class Analyzer( _.transform { // Extracts children expressions of a WindowFunction (input parameters of // a WindowFunction). - case wf: WindowFunction => + case wf : WindowFunction => val newChildren = wf.children.map(extractExpr) wf.withNewChildren(newChildren) 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 7c3d45b1e40c0..5c2aa3c06b3e7 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 @@ -323,13 +323,13 @@ object FunctionRegistry { } else { // Otherwise, find an ctor method that matches the number of arguments, and use that. val params = Seq.fill(expressions.size)(classOf[Expression]) - val f = Try(tag.runtimeClass.getDeclaredConstructor(params: _*)) match { + val f = Try(tag.runtimeClass.getDeclaredConstructor(params : _*)) match { case Success(e) => e case Failure(e) => throw new AnalysisException(s"Invalid number of arguments for function $name") } - Try(f.newInstance(expressions: _*).asInstanceOf[Expression]) match { + Try(f.newInstance(expressions : _*).asInstanceOf[Expression]) match { case Success(e) => e case Failure(e) => throw new AnalysisException(e.getMessage) } 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 75c36d93108df..e9e20670817fe 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 @@ -529,7 +529,7 @@ object HiveTypeCoercion { if falseValues.contains(value) => And(IsNotNull(bool), Not(bool)) case EqualTo(left @ BooleanType(), right @ NumericType()) => - transform(left, right) + transform(left , right) case EqualTo(left @ NumericType(), right @ BooleanType()) => transform(right, left) case EqualNullSafe(left @ BooleanType(), right @ NumericType()) => 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 c4dbcb7b60628..5ac1984043d87 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 @@ -61,11 +61,9 @@ package object dsl { trait ImplicitOperators { def expr: Expression - // scalastyle:off whitespacebeforetoken def unary_- : Expression = UnaryMinus(expr) def unary_! : Predicate = Not(expr) def unary_~ : Expression = BitwiseNot(expr) - // scalastyle:on whitespacebeforetoken def + (other: Expression): Expression = Add(expr, other) def - (other: Expression): Expression = Subtract(expr, other) @@ -143,7 +141,7 @@ package object dsl { // Note that if we make ExpressionConversions an object rather than a trait, we can // then make this a value class to avoid the small penalty of runtime instantiation. def $(args: Any*): analysis.UnresolvedAttribute = { - analysis.UnresolvedAttribute(sc.s(args: _*)) + analysis.UnresolvedAttribute(sc.s(args : _*)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index fa4c2d93eccec..05f746e72b498 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -44,7 +44,7 @@ import org.apache.spark.util.Utils * to the name `value`. */ object ExpressionEncoder { - def apply[T: TypeTag](): ExpressionEncoder[T] = { + def apply[T : TypeTag](): ExpressionEncoder[T] = { // We convert the not-serializable TypeTag into StructType and ClassTag. val mirror = typeTag[T].mirror val cls = mirror.runtimeClass(typeTag[T].tpe) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala index 08ada1f38ba96..9e283f5eb6342 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala @@ -27,7 +27,7 @@ package object encoders { * references from a specific schema.) This requirement allows us to preserve whether a given * object type is being bound by name or by ordinal when doing resolution. */ - private[sql] def encoderFor[A: Encoder]: ExpressionEncoder[A] = implicitly[Encoder[A]] match { + private[sql] def encoderFor[A : Encoder]: ExpressionEncoder[A] = implicitly[Encoder[A]] match { case e: ExpressionEncoder[A] => e.assertUnresolved() e 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 4ffbfa57e726d..d6219514b752b 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 @@ -164,7 +164,7 @@ abstract class Expression extends TreeNode[Expression] { * Returns the hash for this expression. Expressions that compute the same result, even if * they differ cosmetically should return the same hash. */ - def semanticHash(): Int = { + def semanticHash() : Int = { def computeHash(e: Seq[Any]): Int = { // See http://stackoverflow.com/questions/113511/hash-code-implementation var hash: Int = 17 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index bf41f85f79096..931f752b4dc1a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -46,7 +46,7 @@ case class Concat(children: Seq[Expression]) extends Expression with ImplicitCas override def eval(input: InternalRow): Any = { val inputs = children.map(_.eval(input).asInstanceOf[UTF8String]) - UTF8String.concat(inputs: _*) + UTF8String.concat(inputs : _*) } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { @@ -99,7 +99,7 @@ case class ConcatWs(children: Seq[Expression]) case null => Iterator(null.asInstanceOf[UTF8String]) } } - UTF8String.concatWs(flatInputs.head, flatInputs.tail: _*) + UTF8String.concatWs(flatInputs.head, flatInputs.tail : _*) } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { @@ -990,7 +990,7 @@ case class FormatNumber(x: Expression, d: Expression) def typeHelper(p: String): String = { x.dataType match { - case _: DecimalType => s"""$p.toJavaBigDecimal()""" + case _ : DecimalType => s"""$p.toJavaBigDecimal()""" case _ => s"$p" } } 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 5489051e9501b..64957db6b4013 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 @@ -496,7 +496,7 @@ case class MapPartitions[T, U]( /** Factory for constructing new `AppendColumn` nodes. */ object AppendColumns { - def apply[T, U: Encoder]( + def apply[T, U : Encoder]( func: T => U, tEncoder: ExpressionEncoder[T], child: LogicalPlan): AppendColumns[T, U] = { @@ -522,7 +522,7 @@ case class AppendColumns[T, U]( /** Factory for constructing new `MapGroups` nodes. */ object MapGroups { - def apply[K, T, U: Encoder]( + def apply[K, T, U : Encoder]( func: (K, Iterator[T]) => TraversableOnce[U], kEncoder: ExpressionEncoder[K], tEncoder: ExpressionEncoder[T], @@ -557,7 +557,7 @@ case class MapGroups[K, T, U]( /** Factory for constructing new `CoGroup` nodes. */ object CoGroup { - def apply[Key, Left, Right, Result: Encoder]( + def apply[Key, Left, Right, Result : Encoder]( func: (Key, Iterator[Left], Iterator[Right]) => TraversableOnce[Result], keyEnc: ExpressionEncoder[Key], leftEnc: ExpressionEncoder[Left], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala index e4417e0955143..9fefc5656aac0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala @@ -122,7 +122,7 @@ object NumberConverter { * unsigned, otherwise it is signed. * NB: This logic is borrowed from org.apache.hadoop.hive.ql.ud.UDFConv */ - def convert(n: Array[Byte], fromBase: Int, toBase: Int ): UTF8String = { + def convert(n: Array[Byte] , fromBase: Int, toBase: Int ): UTF8String = { if (fromBase < Character.MIN_RADIX || fromBase > Character.MAX_RADIX || Math.abs(toBase) < Character.MIN_RADIX || Math.abs(toBase) > Character.MAX_RADIX) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala index be7573b95d841..520e344361625 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala @@ -90,7 +90,7 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT private[sql] lazy val interpretedOrdering: Ordering[ArrayData] = new Ordering[ArrayData] { private[this] val elementOrdering: Ordering[Any] = elementType match { case dt: AtomicType => dt.ordering.asInstanceOf[Ordering[Any]] - case a: ArrayType => a.interpretedOrdering.asInstanceOf[Ordering[Any]] + case a : ArrayType => a.interpretedOrdering.asInstanceOf[Ordering[Any]] case s: StructType => s.interpretedOrdering.asInstanceOf[Ordering[Any]] case other => throw new IllegalArgumentException(s"Type $other does not support ordered operations") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 864b47a2a08aa..38ce1604b1ede 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -310,7 +310,6 @@ final class Decimal extends Ordered[Decimal] with Serializable { def remainder(that: Decimal): Decimal = this % that - // scalastyle:off whitespacebeforetoken def unary_- : Decimal = { if (decimalVal.ne(null)) { Decimal(-decimalVal, precision, scale) @@ -318,7 +317,6 @@ final class Decimal extends Ordered[Decimal] with Serializable { Decimal(-longVal, precision, scale) } } - // scalastyle:on whitespacebeforetoken def abs: Decimal = if (this.compare(Decimal.ZERO) < 0) this.unary_- else this diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala index a1c4a861c610f..8c766ef829923 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala @@ -98,5 +98,5 @@ class EncoderErrorMessageSuite extends SparkFunSuite { s"""array element class: "${clsName[NonEncodable]}"""")) } - private def clsName[T: ClassTag]: String = implicitly[ClassTag[T]].runtimeClass.getName + private def clsName[T : ClassTag]: String = implicitly[ClassTag[T]].runtimeClass.getName } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 67f4dc98be231..88c558d80a79a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -80,7 +80,7 @@ class JavaSerializable(val value: Int) extends Serializable { class ExpressionEncoderSuite extends SparkFunSuite { OuterScopes.outerScopes.put(getClass.getName, this) - implicit def encoder[T: TypeTag]: ExpressionEncoder[T] = ExpressionEncoder() + implicit def encoder[T : TypeTag]: ExpressionEncoder[T] = ExpressionEncoder() // test flat encoders encodeDecodeTest(false, "primitive boolean") @@ -145,7 +145,7 @@ class ExpressionEncoderSuite extends SparkFunSuite { encoderFor(Encoders.javaSerialization[JavaSerializable])) // test product encoders - private def productTest[T <: Product: ExpressionEncoder](input: T): Unit = { + private def productTest[T <: Product : ExpressionEncoder](input: T): Unit = { encodeDecodeTest(input, input.getClass.getSimpleName) } @@ -286,7 +286,7 @@ class ExpressionEncoderSuite extends SparkFunSuite { } } - private def encodeDecodeTest[T: ExpressionEncoder]( + private def encodeDecodeTest[T : ExpressionEncoder]( input: T, testName: String): Unit = { test(s"encode/decode for $testName: $input") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index 6932f185b9d62..000a3b7ecb7c6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -80,7 +80,7 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { checkCondition(('a < 2 || 'a > 3 || 'b > 5) && 'a < 2, 'a < 2) - checkCondition('a < 2 && ('a < 2 || 'a > 3 || 'b > 5), 'a < 2) + checkCondition('a < 2 && ('a < 2 || 'a > 3 || 'b > 5) , 'a < 2) checkCondition(('a < 2 || 'b > 3) && ('a < 2 || 'c > 5), 'a < 2 || ('b > 3 && 'c > 5)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index a434d03332459..e8c61d6e01dc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -152,7 +152,7 @@ class Column(protected[sql] val expr: Expression) extends Logging { * results into the correct JVM types. * @since 1.6.0 */ - def as[U: Encoder]: TypedColumn[Any, U] = new TypedColumn[Any, U](expr, encoderFor[U]) + def as[U : Encoder]: TypedColumn[Any, U] = new TypedColumn[Any, U](expr, encoderFor[U]) /** * Extracts a value or values from a complex type. @@ -171,7 +171,6 @@ class Column(protected[sql] val expr: Expression) extends Logging { UnresolvedExtractValue(expr, lit(extraction).expr) } - // scalastyle:off whitespacebeforetoken /** * Unary minus, i.e. negate the expression. * {{{ @@ -203,7 +202,6 @@ class Column(protected[sql] val expr: Expression) extends Logging { * @since 1.3.0 */ def unary_! : Column = withExpr { Not(expr) } - // scalastyle:on whitespacebeforetoken /** * Equality test. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index fac8950aee12d..60d2f05b8605b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -204,7 +204,7 @@ class DataFrame private[sql]( * @since 1.6.0 */ @Experimental - def as[U: Encoder]: Dataset[U] = new Dataset[U](sqlContext, logicalPlan) + def as[U : Encoder]: Dataset[U] = new Dataset[U](sqlContext, logicalPlan) /** * Returns a new [[DataFrame]] with columns renamed. This can be quite convenient in conversion @@ -227,7 +227,7 @@ class DataFrame private[sql]( val newCols = logicalPlan.output.zip(colNames).map { case (oldAttribute, newName) => Column(oldAttribute).as(newName) } - select(newCols: _*) + select(newCols : _*) } /** @@ -579,7 +579,7 @@ class DataFrame private[sql]( */ @scala.annotation.varargs def sortWithinPartitions(sortCol: String, sortCols: String*): DataFrame = { - sortWithinPartitions((sortCol +: sortCols).map(Column(_)): _*) + sortWithinPartitions((sortCol +: sortCols).map(Column(_)) : _*) } /** @@ -608,7 +608,7 @@ class DataFrame private[sql]( */ @scala.annotation.varargs def sort(sortCol: String, sortCols: String*): DataFrame = { - sort((sortCol +: sortCols).map(apply): _*) + sort((sortCol +: sortCols).map(apply) : _*) } /** @@ -631,7 +631,7 @@ class DataFrame private[sql]( * @since 1.3.0 */ @scala.annotation.varargs - def orderBy(sortCol: String, sortCols: String*): DataFrame = sort(sortCol, sortCols: _*) + def orderBy(sortCol: String, sortCols: String*): DataFrame = sort(sortCol, sortCols : _*) /** * Returns a new [[DataFrame]] sorted by the given expressions. @@ -640,7 +640,7 @@ class DataFrame private[sql]( * @since 1.3.0 */ @scala.annotation.varargs - def orderBy(sortExprs: Column*): DataFrame = sort(sortExprs: _*) + def orderBy(sortExprs: Column*): DataFrame = sort(sortExprs : _*) /** * Selects column based on the column name and return it as a [[Column]]. @@ -720,7 +720,7 @@ class DataFrame private[sql]( * @since 1.3.0 */ @scala.annotation.varargs - def select(col: String, cols: String*): DataFrame = select((col +: cols).map(Column(_)): _*) + def select(col: String, cols: String*): DataFrame = select((col +: cols).map(Column(_)) : _*) /** * Selects a set of SQL expressions. This is a variant of `select` that accepts @@ -948,7 +948,7 @@ class DataFrame private[sql]( * @since 1.3.0 */ def agg(aggExpr: (String, String), aggExprs: (String, String)*): DataFrame = { - groupBy().agg(aggExpr, aggExprs: _*) + groupBy().agg(aggExpr, aggExprs : _*) } /** @@ -986,7 +986,7 @@ class DataFrame private[sql]( * @since 1.3.0 */ @scala.annotation.varargs - def agg(expr: Column, exprs: Column*): DataFrame = groupBy().agg(expr, exprs: _*) + def agg(expr: Column, exprs: Column*): DataFrame = groupBy().agg(expr, exprs : _*) /** * Returns a new [[DataFrame]] by taking the first `n` rows. The difference between this function @@ -1118,7 +1118,7 @@ class DataFrame private[sql]( * @group dfops * @since 1.3.0 */ - def explode[A <: Product: TypeTag](input: Column*)(f: Row => TraversableOnce[A]): DataFrame = { + def explode[A <: Product : TypeTag](input: Column*)(f: Row => TraversableOnce[A]): DataFrame = { val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val elementTypes = schema.toAttributes.map { @@ -1147,7 +1147,7 @@ class DataFrame private[sql]( * @group dfops * @since 1.3.0 */ - def explode[A, B: TypeTag](inputColumn: String, outputColumn: String)(f: A => TraversableOnce[B]) + def explode[A, B : TypeTag](inputColumn: String, outputColumn: String)(f: A => TraversableOnce[B]) : DataFrame = { val dataType = ScalaReflection.schemaFor[B].dataType val attributes = AttributeReference(outputColumn, dataType)() :: Nil @@ -1186,7 +1186,7 @@ class DataFrame private[sql]( Column(field) } } - select(columns: _*) + select(columns : _*) } else { select(Column("*"), col.as(colName)) } @@ -1207,7 +1207,7 @@ class DataFrame private[sql]( Column(field) } } - select(columns: _*) + select(columns : _*) } else { select(Column("*"), col.as(colName, metadata)) } @@ -1231,7 +1231,7 @@ class DataFrame private[sql]( Column(col) } } - select(columns: _*) + select(columns : _*) } else { this } @@ -1244,7 +1244,7 @@ class DataFrame private[sql]( * @since 1.4.0 */ def drop(colName: String): DataFrame = { - drop(Seq(colName): _*) + drop(Seq(colName) : _*) } /** @@ -1283,7 +1283,7 @@ class DataFrame private[sql]( val colsAfterDrop = attrs.filter { attr => attr != expression }.map(attr => Column(attr)) - select(colsAfterDrop: _*) + select(colsAfterDrop : _*) } /** @@ -1479,7 +1479,7 @@ class DataFrame private[sql]( * @group action * @since 1.6.0 */ - def takeAsList(n: Int): java.util.List[Row] = java.util.Arrays.asList(take(n): _*) + def takeAsList(n: Int): java.util.List[Row] = java.util.Arrays.asList(take(n) : _*) /** * Returns an array that contains all of [[Row]]s in this [[DataFrame]]. @@ -1505,7 +1505,7 @@ class DataFrame private[sql]( */ def collectAsList(): java.util.List[Row] = withCallback("collectAsList", this) { _ => withNewExecutionId { - java.util.Arrays.asList(rdd.collect(): _*) + java.util.Arrays.asList(rdd.collect() : _*) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameHolder.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameHolder.scala index 4441a634be407..3b30337f1f877 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameHolder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameHolder.scala @@ -33,5 +33,5 @@ case class DataFrameHolder private[sql](private val df: DataFrame) { // `rdd.toDF("1")` as invoking this toDF and then apply on the returned DataFrame. def toDF(): DataFrame = df - def toDF(colNames: String*): DataFrame = df.toDF(colNames: _*) + def toDF(colNames: String*): DataFrame = df.toDF(colNames : _*) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 43500b09e0f38..f7be5f6b370ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -164,7 +164,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { df.col(f.name) } } - df.select(projections: _*) + df.select(projections : _*) } /** @@ -191,7 +191,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { df.col(f.name) } } - df.select(projections: _*) + df.select(projections : _*) } /** @@ -364,7 +364,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { df.col(f.name) } } - df.select(projections: _*) + df.select(projections : _*) } private def fill0(values: Seq[(String, Any)]): DataFrame = { @@ -395,7 +395,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { } }.getOrElse(df.col(f.name)) } - df.select(projections: _*) + df.select(projections : _*) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 1ed451d5a8bab..d948e4894253c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -203,7 +203,7 @@ class DataFrameReader private[sql](sqlContext: SQLContext) extends Logging { predicates: Array[String], connectionProperties: Properties): DataFrame = { val parts: Array[Partition] = predicates.zipWithIndex.map { case (part, i) => - JDBCPartition(part, i): Partition + JDBCPartition(part, i) : Partition } jdbc(url, table, parts, connectionProperties) } @@ -262,7 +262,7 @@ class DataFrameReader private[sql](sqlContext: SQLContext) extends Logging { * * @since 1.6.0 */ - def json(paths: String*): DataFrame = format("json").load(paths: _*) + def json(paths: String*): DataFrame = format("json").load(paths : _*) /** * Loads an `JavaRDD[String]` storing JSON objects (one object per record) and @@ -355,7 +355,7 @@ class DataFrameReader private[sql](sqlContext: SQLContext) extends Logging { * @since 1.6.0 */ @scala.annotation.varargs - def text(paths: String*): DataFrame = format("text").load(paths: _*) + def text(paths: String*): DataFrame = format("text").load(paths : _*) /////////////////////////////////////////////////////////////////////////////////////// // Builder pattern config options diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 9ffb5b94b2d18..42f01e9359c64 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -131,7 +131,7 @@ class Dataset[T] private[sql]( * along with `alias` or `as` to rearrange or rename as required. * @since 1.6.0 */ - def as[U: Encoder]: Dataset[U] = { + def as[U : Encoder]: Dataset[U] = { new Dataset(sqlContext, queryExecution, encoderFor[U]) } @@ -318,7 +318,7 @@ class Dataset[T] private[sql]( * Returns a new [[Dataset]] that contains the result of applying `func` to each element. * @since 1.6.0 */ - def map[U: Encoder](func: T => U): Dataset[U] = mapPartitions(_.map(func)) + def map[U : Encoder](func: T => U): Dataset[U] = mapPartitions(_.map(func)) /** * (Java-specific) @@ -333,7 +333,7 @@ class Dataset[T] private[sql]( * Returns a new [[Dataset]] that contains the result of applying `func` to each partition. * @since 1.6.0 */ - def mapPartitions[U: Encoder](func: Iterator[T] => Iterator[U]): Dataset[U] = { + def mapPartitions[U : Encoder](func: Iterator[T] => Iterator[U]): Dataset[U] = { new Dataset[U]( sqlContext, MapPartitions[T, U]( @@ -360,7 +360,7 @@ class Dataset[T] private[sql]( * and then flattening the results. * @since 1.6.0 */ - def flatMap[U: Encoder](func: T => TraversableOnce[U]): Dataset[U] = + def flatMap[U : Encoder](func: T => TraversableOnce[U]): Dataset[U] = mapPartitions(_.flatMap(func)) /** @@ -432,7 +432,7 @@ class Dataset[T] private[sql]( * Returns a [[GroupedDataset]] where the data is grouped by the given key `func`. * @since 1.6.0 */ - def groupBy[K: Encoder](func: T => K): GroupedDataset[K, T] = { + def groupBy[K : Encoder](func: T => K): GroupedDataset[K, T] = { val inputPlan = logicalPlan val withGroupingKey = AppendColumns(func, resolvedTEncoder, inputPlan) val executed = sqlContext.executePlan(withGroupingKey) @@ -566,14 +566,14 @@ class Dataset[T] private[sql]( * Returns a new [[Dataset]] by sampling a fraction of records. * @since 1.6.0 */ - def sample(withReplacement: Boolean, fraction: Double, seed: Long): Dataset[T] = + def sample(withReplacement: Boolean, fraction: Double, seed: Long) : Dataset[T] = withPlan(Sample(0.0, fraction, withReplacement, seed, _)) /** * Returns a new [[Dataset]] by sampling a fraction of records, using a random seed. * @since 1.6.0 */ - def sample(withReplacement: Boolean, fraction: Double): Dataset[T] = { + def sample(withReplacement: Boolean, fraction: Double) : Dataset[T] = { sample(withReplacement, fraction, Utils.random.nextLong) } @@ -731,7 +731,7 @@ class Dataset[T] private[sql]( * a very large `num` can crash the driver process with OutOfMemoryError. * @since 1.6.0 */ - def takeAsList(num: Int): java.util.List[T] = java.util.Arrays.asList(take(num): _*) + def takeAsList(num: Int): java.util.List[T] = java.util.Arrays.asList(take(num) : _*) /** * Persist this [[Dataset]] with the default storage level (`MEMORY_AND_DISK`). @@ -786,7 +786,7 @@ class Dataset[T] private[sql]( private[sql] def withPlan(f: LogicalPlan => LogicalPlan): Dataset[T] = new Dataset[T](sqlContext, sqlContext.executePlan(f(logicalPlan)), tEncoder) - private[sql] def withPlan[R: Encoder]( + private[sql] def withPlan[R : Encoder]( other: Dataset[_])( f: (LogicalPlan, LogicalPlan) => LogicalPlan): Dataset[R] = new Dataset[R](sqlContext, f(logicalPlan, other.logicalPlan)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala index f5cbf013bce9d..c74ef2c03541e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala @@ -229,7 +229,7 @@ class GroupedData protected[sql]( */ @scala.annotation.varargs def mean(colNames: String*): DataFrame = { - aggregateNumericColumns(colNames: _*)(Average) + aggregateNumericColumns(colNames : _*)(Average) } /** @@ -241,7 +241,7 @@ class GroupedData protected[sql]( */ @scala.annotation.varargs def max(colNames: String*): DataFrame = { - aggregateNumericColumns(colNames: _*)(Max) + aggregateNumericColumns(colNames : _*)(Max) } /** @@ -253,7 +253,7 @@ class GroupedData protected[sql]( */ @scala.annotation.varargs def avg(colNames: String*): DataFrame = { - aggregateNumericColumns(colNames: _*)(Average) + aggregateNumericColumns(colNames : _*)(Average) } /** @@ -265,7 +265,7 @@ class GroupedData protected[sql]( */ @scala.annotation.varargs def min(colNames: String*): DataFrame = { - aggregateNumericColumns(colNames: _*)(Min) + aggregateNumericColumns(colNames : _*)(Min) } /** @@ -277,7 +277,7 @@ class GroupedData protected[sql]( */ @scala.annotation.varargs def sum(colNames: String*): DataFrame = { - aggregateNumericColumns(colNames: _*)(Sum) + aggregateNumericColumns(colNames : _*)(Sum) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala index 12179367fa012..a819ddceb1b1b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala @@ -73,7 +73,7 @@ class GroupedDataset[K, V] private[sql]( * * @since 1.6.0 */ - def keyAs[L: Encoder]: GroupedDataset[L, V] = + def keyAs[L : Encoder]: GroupedDataset[L, V] = new GroupedDataset( encoderFor[L], unresolvedVEncoder, @@ -110,7 +110,7 @@ class GroupedDataset[K, V] private[sql]( * * @since 1.6.0 */ - def flatMapGroups[U: Encoder](f: (K, Iterator[V]) => TraversableOnce[U]): Dataset[U] = { + def flatMapGroups[U : Encoder](f: (K, Iterator[V]) => TraversableOnce[U]): Dataset[U] = { new Dataset[U]( sqlContext, MapGroups( @@ -158,7 +158,7 @@ class GroupedDataset[K, V] private[sql]( * * @since 1.6.0 */ - def mapGroups[U: Encoder](f: (K, Iterator[V]) => U): Dataset[U] = { + def mapGroups[U : Encoder](f: (K, Iterator[V]) => U): Dataset[U] = { val func = (key: K, it: Iterator[V]) => Iterator(f(key, it)) flatMapGroups(func) } @@ -302,7 +302,7 @@ class GroupedDataset[K, V] private[sql]( * * @since 1.6.0 */ - def cogroup[U, R: Encoder]( + def cogroup[U, R : Encoder]( other: GroupedDataset[K, U])( f: (K, Iterator[V], Iterator[U]) => TraversableOnce[R]): Dataset[R] = { new Dataset[R]( 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 6721d9c40748b..2dd82358fbfdf 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 @@ -409,7 +409,7 @@ class SQLContext private[sql]( * @since 1.3.0 */ @Experimental - def createDataFrame[A <: Product: TypeTag](rdd: RDD[A]): DataFrame = { + def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { SQLContext.setActive(self) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val attributeSeq = schema.toAttributes @@ -425,7 +425,7 @@ class SQLContext private[sql]( * @since 1.3.0 */ @Experimental - def createDataFrame[A <: Product: TypeTag](data: Seq[A]): DataFrame = { + def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { SQLContext.setActive(self) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val attributeSeq = schema.toAttributes @@ -498,7 +498,7 @@ class SQLContext private[sql]( } - def createDataset[T: Encoder](data: Seq[T]): Dataset[T] = { + def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = { val enc = encoderFor[T] val attributes = enc.schema.toAttributes val encoded = data.map(d => enc.toRow(d).copy()) @@ -507,7 +507,7 @@ class SQLContext private[sql]( new Dataset[T](this, plan) } - def createDataset[T: Encoder](data: RDD[T]): Dataset[T] = { + def createDataset[T : Encoder](data: RDD[T]): Dataset[T] = { val enc = encoderFor[T] val attributes = enc.schema.toAttributes val encoded = data.map(d => enc.toRow(d)) @@ -516,7 +516,7 @@ class SQLContext private[sql]( new Dataset[T](this, plan) } - def createDataset[T: Encoder](data: java.util.List[T]): Dataset[T] = { + def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = { createDataset(data.asScala) } @@ -945,7 +945,7 @@ class SQLContext private[sql]( } } - // Register a successfully instantiated context to the singleton. This should be at the end of + // Register a succesfully instantiatd context to the singleton. This should be at the end of // the class definition so that the singleton is updated only if there is no exception in the // construction of the instance. sparkContext.addSparkListener(new SparkListener { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala index a7f7997df1a8b..ab414799f1a42 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala @@ -37,7 +37,7 @@ abstract class SQLImplicits { protected def _sqlContext: SQLContext /** @since 1.6.0 */ - implicit def newProductEncoder[T <: Product: TypeTag]: Encoder[T] = ExpressionEncoder() + implicit def newProductEncoder[T <: Product : TypeTag]: Encoder[T] = ExpressionEncoder() /** @since 1.6.0 */ implicit def newIntEncoder: Encoder[Int] = ExpressionEncoder() @@ -67,7 +67,7 @@ abstract class SQLImplicits { * Creates a [[Dataset]] from an RDD. * @since 1.6.0 */ - implicit def rddToDatasetHolder[T: Encoder](rdd: RDD[T]): DatasetHolder[T] = { + implicit def rddToDatasetHolder[T : Encoder](rdd: RDD[T]): DatasetHolder[T] = { DatasetHolder(_sqlContext.createDataset(rdd)) } @@ -75,7 +75,7 @@ abstract class SQLImplicits { * Creates a [[Dataset]] from a local Seq. * @since 1.6.0 */ - implicit def localSeqToDatasetHolder[T: Encoder](s: Seq[T]): DatasetHolder[T] = { + implicit def localSeqToDatasetHolder[T : Encoder](s: Seq[T]): DatasetHolder[T] = { DatasetHolder(_sqlContext.createDataset(s)) } @@ -89,7 +89,7 @@ abstract class SQLImplicits { * Creates a DataFrame from an RDD of Product (e.g. case classes, tuples). * @since 1.3.0 */ - implicit def rddToDataFrameHolder[A <: Product: TypeTag](rdd: RDD[A]): DataFrameHolder = { + implicit def rddToDataFrameHolder[A <: Product : TypeTag](rdd: RDD[A]): DataFrameHolder = { DataFrameHolder(_sqlContext.createDataFrame(rdd)) } @@ -97,7 +97,7 @@ abstract class SQLImplicits { * Creates a DataFrame from a local Seq of Product. * @since 1.3.0 */ - implicit def localSeqToDataFrameHolder[A <: Product: TypeTag](data: Seq[A]): DataFrameHolder = + implicit def localSeqToDataFrameHolder[A <: Product : TypeTag](data: Seq[A]): DataFrameHolder = { DataFrameHolder(_sqlContext.createDataFrame(data)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index a8e6a40169d81..d912aeb70d517 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -39,7 +39,7 @@ private[r] object SQLUtils { new JavaSparkContext(sqlCtx.sparkContext) } - def createStructType(fields: Seq[StructField]): StructType = { + def createStructType(fields : Seq[StructField]): StructType = { StructType(fields) } 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 058d147c7d65d..6b100577077c6 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 @@ -223,7 +223,7 @@ case class Exchange( new ShuffledRowRDD(shuffleDependency, specifiedPartitionStartIndices) } - protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") { + protected override def doExecute(): RDD[InternalRow] = attachTree(this , "execute") { coordinator match { case Some(exchangeCoordinator) => val shuffleRDD = exchangeCoordinator.postShuffleRDD(this) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Queryable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Queryable.scala index bb551614779b5..38263af0f7e30 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Queryable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Queryable.scala @@ -71,7 +71,7 @@ private[sql] trait Queryable { private[sql] def formatString ( rows: Seq[Seq[String]], numRows: Int, - hasMoreData: Boolean, + hasMoreData : Boolean, truncate: Boolean = true): String = { val sb = new StringBuilder val numCols = schema.fieldNames.length diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala index b5ac530444b79..1df38f7ff59cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.types._ object TypedAggregateExpression { - def apply[A, B: Encoder, C: Encoder]( + def apply[A, B : Encoder, C : Encoder]( aggregator: Aggregator[A, B, C]): TypedAggregateExpression = { new TypedAggregateExpression( aggregator.asInstanceOf[Aggregator[Any, Any, Any]], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala index d5e0d80076cbe..d45d2db62f3a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala @@ -256,7 +256,7 @@ private[spark] class SqlNewHadoopRDD[V: ClassTag]( val infos = c.newGetLocationInfo.invoke(split).asInstanceOf[Array[AnyRef]] Some(HadoopRDD.convertSplitLocationInfo(infos)) } catch { - case e: Exception => + case e : Exception => logDebug("Failed to use InputSplit#getLocationInfo.", e) None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala index c4b125e9d5f00..fb97a03df60f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala @@ -557,7 +557,7 @@ private[parquet] object CatalystSchemaConverter { } } - private def computeMinBytesForPrecision(precision: Int): Int = { + private def computeMinBytesForPrecision(precision : Int) : Int = { var numBytes = 1 while (math.pow(2.0, 8 * numBytes - 1) < math.pow(10.0, precision)) { numBytes += 1 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 index a567457dba3c5..93d32e1fb93ae 100644 --- 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 @@ -34,7 +34,7 @@ import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter * materialize the right RDD (in case of the right RDD is nondeterministic). */ private[spark] -class UnsafeCartesianRDD(left: RDD[UnsafeRow], right: RDD[UnsafeRow], numFieldsOfRight: Int) +class UnsafeCartesianRDD(left : RDD[UnsafeRow], right : RDD[UnsafeRow], numFieldsOfRight: Int) extends CartesianRDD[UnsafeRow, UnsafeRow](left.sparkContext, left, right) { override def compute(split: Partition, context: TaskContext): Iterator[(UnsafeRow, UnsafeRow)] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 8c68d9ee0a1ef..52735c9d7f8c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -64,7 +64,7 @@ private[sql] trait SQLMetricValue[T] extends Serializable { /** * A wrapper of Long to avoid boxing and unboxing when using Accumulator */ -private[sql] class LongSQLMetricValue(private var _value: Long) extends SQLMetricValue[Long] { +private[sql] class LongSQLMetricValue(private var _value : Long) extends SQLMetricValue[Long] { def add(incr: Long): LongSQLMetricValue = { _value += incr diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala index a4cb54e2bf2a2..a191759813de1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala @@ -94,7 +94,7 @@ private[sql] object FrequentItems extends Logging { (name, originalSchema.fields(index).dataType) }.toArray - val freqItems = df.select(cols.map(Column(_)): _*).rdd.aggregate(countMaps)( + val freqItems = df.select(cols.map(Column(_)) : _*).rdd.aggregate(countMaps)( seqOp = (counts, row) => { var i = 0 while (i < numCols) { @@ -115,7 +115,7 @@ private[sql] object FrequentItems extends Logging { } ) val justItems = freqItems.map(m => m.baseMap.keys.toArray) - val resultRow = Row(justItems: _*) + val resultRow = Row(justItems : _*) // append frequent Items to the column name for easy debugging val outputCols = colInfo.map { v => StructField(v._1 + "_freqItems", ArrayType(v._2, false)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala index 05a9f377b9897..e9b60841fc28c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala @@ -44,7 +44,7 @@ object Window { */ @scala.annotation.varargs def partitionBy(colName: String, colNames: String*): WindowSpec = { - spec.partitionBy(colName, colNames: _*) + spec.partitionBy(colName, colNames : _*) } /** @@ -53,7 +53,7 @@ object Window { */ @scala.annotation.varargs def partitionBy(cols: Column*): WindowSpec = { - spec.partitionBy(cols: _*) + spec.partitionBy(cols : _*) } /** @@ -62,7 +62,7 @@ object Window { */ @scala.annotation.varargs def orderBy(colName: String, colNames: String*): WindowSpec = { - spec.orderBy(colName, colNames: _*) + spec.orderBy(colName, colNames : _*) } /** @@ -71,7 +71,7 @@ object Window { */ @scala.annotation.varargs def orderBy(cols: Column*): WindowSpec = { - spec.orderBy(cols: _*) + spec.orderBy(cols : _*) } private def spec: WindowSpec = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 1ac62883a68ee..592d79df3109a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -306,7 +306,7 @@ object functions extends LegacyFunctions { */ @scala.annotation.varargs def countDistinct(columnName: String, columnNames: String*): Column = - countDistinct(Column(columnName), columnNames.map(Column.apply): _*) + countDistinct(Column(columnName), columnNames.map(Column.apply) : _*) /** * Aggregate function: returns the first value in a group. @@ -768,7 +768,7 @@ object functions extends LegacyFunctions { */ @scala.annotation.varargs def array(colName: String, colNames: String*): Column = { - array((colName +: colNames).map(col): _*) + array((colName +: colNames).map(col) : _*) } /** @@ -977,7 +977,7 @@ object functions extends LegacyFunctions { */ @scala.annotation.varargs def struct(colName: String, colNames: String*): Column = { - struct((colName +: colNames).map(col): _*) + struct((colName +: colNames).map(col) : _*) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala index d2c31d6e04107..467d8d62d1b7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala @@ -30,7 +30,7 @@ private class AggregatedDialect(dialects: List[JdbcDialect]) extends JdbcDialect require(dialects.nonEmpty) - override def canHandle(url: String): Boolean = + override def canHandle(url : String): Boolean = dialects.map(_.canHandle(url)).reduce(_ && _) override def getCatalystType( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index 8d58321d4887d..ca2d909e2cccc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.types._ * send a null value to the database. */ @DeveloperApi -case class JdbcType(databaseTypeDefinition: String, jdbcNullType: Int) +case class JdbcType(databaseTypeDefinition : String, jdbcNullType : Int) /** * :: DeveloperApi :: @@ -60,7 +60,7 @@ abstract class JdbcDialect extends Serializable { * @return True if the dialect can be applied on the given jdbc url. * @throws NullPointerException if the url is null. */ - def canHandle(url: String): Boolean + def canHandle(url : String): Boolean /** * Get the custom datatype mapping for the given jdbc meta information. @@ -130,7 +130,7 @@ object JdbcDialects { * * @param dialect The new dialect. */ - def registerDialect(dialect: JdbcDialect): Unit = { + def registerDialect(dialect: JdbcDialect) : Unit = { dialects = dialect :: dialects.filterNot(_ == dialect) } @@ -139,7 +139,7 @@ object JdbcDialects { * * @param dialect The jdbc dialect. */ - def unregisterDialect(dialect: JdbcDialect): Unit = { + def unregisterDialect(dialect : JdbcDialect) : Unit = { dialects = dialects.filterNot(_ == dialect) } @@ -169,5 +169,5 @@ object JdbcDialects { * NOOP dialect object, always returning the neutral element. */ private object NoopDialect extends JdbcDialect { - override def canHandle(url: String): Boolean = true + override def canHandle(url : String): Boolean = true } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala index faae54e605c68..e1717049f383d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala @@ -23,13 +23,10 @@ import org.apache.spark.sql.types.{BooleanType, DataType, LongType, MetadataBuil private case object MySQLDialect extends JdbcDialect { - override def canHandle(url: String): Boolean = url.startsWith("jdbc:mysql") + override def canHandle(url : String): Boolean = url.startsWith("jdbc:mysql") override def getCatalystType( - sqlType: Int, - typeName: String, - size: Int, - md: MetadataBuilder): Option[DataType] = { + sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = { if (sqlType == Types.VARBINARY && typeName.equals("BIT") && size != 1) { // This could instead be a BinaryType if we'd rather return bit-vectors of up to 64 bits as // byte arrays instead of longs. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala index f952fc07fd387..3258f3782d8cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext /** An `Aggregator` that adds up any numeric type returned by the given function. */ -class SumOf[I, N: Numeric](f: I => N) extends Aggregator[I, N, N] { +class SumOf[I, N : Numeric](f: I => N) extends Aggregator[I, N, N] { val numeric = implicitly[Numeric[N]] override def zero: N = numeric.zero @@ -113,7 +113,7 @@ class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { import testImplicits._ - def sum[I, N: Numeric: Encoder](f: I => N): TypedColumn[I, N] = + def sum[I, N : Numeric : Encoder](f: I => N): TypedColumn[I, N] = new SumOf(f).toColumn test("typed aggregation: TypedAggregator") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala index 848f1af65508b..3a283a4e1f610 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala @@ -27,7 +27,7 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("persist and unpersist") { - val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS().select(expr("_2 + 1").as[Int]) + val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS().select(expr("_2 + 1").as[Int]) val cached = ds.cache() // count triggers the caching action. It should not throw. cached.count() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index a3ed2e06165ea..53b5f45c2d4a6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -30,7 +30,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("toDS") { - val data = Seq(("a", 1), ("b", 2), ("c", 3)) + val data = Seq(("a", 1) , ("b", 2), ("c", 3)) checkAnswer( data.toDS(), data: _*) @@ -87,7 +87,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("as case class / collect") { - val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDF("a", "b").as[ClassData] + val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDF("a", "b").as[ClassData] checkAnswer( ds, ClassData("a", 1), ClassData("b", 2), ClassData("c", 3)) @@ -105,7 +105,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("map") { - val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() checkAnswer( ds.map(v => (v._1, v._2 + 1)), ("a", 2), ("b", 3), ("c", 4)) @@ -124,23 +124,23 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select") { - val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select(expr("_2 + 1").as[Int]), 2, 3, 4) } test("select 2") { - val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select( expr("_1").as[String], - expr("_2").as[Int]): Dataset[(String, Int)], + expr("_2").as[Int]) : Dataset[(String, Int)], ("a", 1), ("b", 2), ("c", 3)) } test("select 2, primitive and tuple") { - val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select( expr("_1").as[String], @@ -149,7 +149,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select 2, primitive and class") { - val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select( expr("_1").as[String], @@ -158,7 +158,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select 2, primitive and class, fields reordered") { - val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() checkDecoding( ds.select( expr("_1").as[String], @@ -167,28 +167,28 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("filter") { - val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() checkAnswer( ds.filter(_._1 == "b"), ("b", 2)) } test("foreach") { - val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() val acc = sparkContext.accumulator(0) ds.foreach(v => acc += v._2) assert(acc.value == 6) } test("foreachPartition") { - val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() val acc = sparkContext.accumulator(0) ds.foreachPartition(_.foreach(v => acc += v._2)) assert(acc.value == 6) } test("reduce") { - val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() assert(ds.reduce((a, b) => ("sum", a._2 + b._2)) == ("sum", 6)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 860e07c68cef1..4ab148065a476 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -206,7 +206,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { StructType( StructField("f1", IntegerType, true) :: StructField("f2", IntegerType, true) :: Nil), - StructType(StructField("f1", LongType, true) :: Nil), + StructType(StructField("f1", LongType, true) :: Nil) , StructType( StructField("f1", LongType, true) :: StructField("f2", IntegerType, true) :: Nil)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index f2e0a868f4b1a..ab48e971b507a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -72,7 +72,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { /** * Writes `data` to a Parquet file, reads it back and check file contents. */ - protected def checkParquetFile[T <: Product: ClassTag: TypeTag](data: Seq[T]): Unit = { + protected def checkParquetFile[T <: Product : ClassTag: TypeTag](data: Seq[T]): Unit = { withParquetDataFrame(data)(r => checkAnswer(r, data.map(Row.fromTuple))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 984e3fbc05e48..1fa22e2933318 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -46,7 +46,7 @@ class JDBCSuite extends SparkFunSuite val testBytes = Array[Byte](99.toByte, 134.toByte, 135.toByte, 200.toByte, 205.toByte) val testH2Dialect = new JdbcDialect { - override def canHandle(url: String): Boolean = url.startsWith("jdbc:h2") + override def canHandle(url: String) : Boolean = url.startsWith("jdbc:h2") override def getCatalystType( sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = Some(StringType) @@ -489,7 +489,7 @@ class JDBCSuite extends SparkFunSuite test("Aggregated dialects") { val agg = new AggregatedDialect(List(new JdbcDialect { - override def canHandle(url: String): Boolean = url.startsWith("jdbc:h2:") + override def canHandle(url: String) : Boolean = url.startsWith("jdbc:h2:") override def getCatalystType( sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = if (sqlType % 2 == 0) { 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 index d1d8a68f6d196..599294dfbb7d7 100644 --- 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive.thriftserver private[hive] object ReflectionUtils { - def setSuperField(obj: Object, fieldName: String, fieldValue: Object) { + def setSuperField(obj : Object, fieldName: String, fieldValue: Object) { setAncestorField(obj, 1, fieldName, fieldValue) } 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 9f9efe33e12a3..03bc830df2034 100644 --- 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 @@ -325,7 +325,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (ret != 0) { // For analysis exception, only the error is printed out to the console. rc.getException() match { - case e: AnalysisException => + case e : AnalysisException => err.println(s"""Error in query: ${e.getMessage}""") case _ => err.println(rc.getErrorMessage()) } @@ -369,7 +369,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (counter != 0) { responseMsg += s", Fetched $counter row(s)" } - console.printInfo(responseMsg, null) + console.printInfo(responseMsg , null) // Destroy the driver to release all the locks. driver.destroy() } else { 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 c9df3c4a82c88..7a260e72eb459 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 @@ -657,8 +657,8 @@ private[hive] trait HiveInspectors { case DecimalType() => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector case StructType(fields) => ObjectInspectorFactory.getStandardStructObjectInspector( - java.util.Arrays.asList(fields.map(f => f.name): _*), - java.util.Arrays.asList(fields.map(f => toInspector(f.dataType)): _*)) + java.util.Arrays.asList(fields.map(f => f.name) : _*), + java.util.Arrays.asList(fields.map(f => toInspector(f.dataType)) : _*)) } /** @@ -905,8 +905,8 @@ private[hive] trait HiveInspectors { getListTypeInfo(elemType.toTypeInfo) case StructType(fields) => getStructTypeInfo( - java.util.Arrays.asList(fields.map(_.name): _*), - java.util.Arrays.asList(fields.map(_.dataType.toTypeInfo): _*)) + java.util.Arrays.asList(fields.map(_.name) : _*), + java.util.Arrays.asList(fields.map(_.dataType.toTypeInfo) : _*)) case MapType(keyType, valueType, _) => getMapTypeInfo(keyType.toTypeInfo, valueType.toTypeInfo) case BinaryType => binaryTypeInfo 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 912cd41173a2a..56cab1aee89df 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 @@ -181,7 +181,7 @@ private[hive] case class HiveSimpleUDF( val ret = FunctionRegistry.invoke( method, function, - conversionHelper.convertIfNecessary(inputs: _*): _*) + conversionHelper.convertIfNecessary(inputs : _*): _*) unwrap(ret, returnInspector) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala index ad28345a667d0..3b867bbfa1817 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala @@ -118,8 +118,8 @@ class HiveInspectorSuite extends SparkFunSuite with HiveInspectors { case DecimalType() => PrimitiveObjectInspectorFactory.writableHiveDecimalObjectInspector case StructType(fields) => ObjectInspectorFactory.getStandardStructObjectInspector( - java.util.Arrays.asList(fields.map(f => f.name): _*), - java.util.Arrays.asList(fields.map(f => toWritableInspector(f.dataType)): _*)) + java.util.Arrays.asList(fields.map(f => f.name) : _*), + java.util.Arrays.asList(fields.map(f => toWritableInspector(f.dataType)) : _*)) } def checkDataType(dt1: Seq[DataType], dt2: Seq[DataType]): Unit = { 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 40e9c9362cf5e..da7303c791064 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 @@ -154,8 +154,8 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef } 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=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)) From 8ed5f12d2bb408bd37e4156b5f1bad9a6b8c3cb5 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 12 Jan 2016 14:19:53 -0800 Subject: [PATCH 084/110] [SPARK-12724] SQL generation support for persisted data source tables This PR implements SQL generation support for persisted data source tables. A new field `metastoreTableIdentifier: Option[TableIdentifier]` is added to `LogicalRelation`. When a `LogicalRelation` representing a persisted data source relation is created, this field holds the database name and table name of the relation. Author: Cheng Lian Closes #10712 from liancheng/spark-12724-datasources-sql-gen. --- .../scala/org/apache/spark/sql/DataFrame.scala | 2 +- .../datasources/DataSourceStrategy.scala | 16 ++++++++-------- .../execution/datasources/LogicalRelation.scala | 8 +++++--- .../datasources/parquet/ParquetRelation.scala | 10 ++-------- .../spark/sql/execution/datasources/rules.scala | 16 ++++++++-------- .../datasources/parquet/ParquetFilterSuite.scala | 2 +- .../parquet/ParquetPartitionDiscoverySuite.scala | 2 +- .../spark/sql/sources/FilteredScanSuite.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 6 ++++-- .../org/apache/spark/sql/hive/SQLBuilder.scala | 14 +++++--------- .../spark/sql/hive/execution/commands.scala | 2 +- .../spark/sql/hive/LogicalPlanToSQLSuite.scala | 10 ++++++++++ .../sql/hive/MetastoreDataSourcesSuite.scala | 2 +- .../spark/sql/hive/execution/SQLQuerySuite.scala | 2 +- .../spark/sql/hive/orc/OrcFilterSuite.scala | 2 +- .../apache/spark/sql/hive/parquetSuites.scala | 8 ++++---- .../sql/sources/hadoopFsRelationSuites.scala | 2 +- 17 files changed, 55 insertions(+), 51 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 60d2f05b8605b..91bf2f8ce4d2f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -1728,7 +1728,7 @@ class DataFrame private[sql]( */ def inputFiles: Array[String] = { val files: Seq[String] = logicalPlan.collect { - case LogicalRelation(fsBasedRelation: FileRelation, _) => + case LogicalRelation(fsBasedRelation: FileRelation, _, _) => fsBasedRelation.inputFiles case fr: FileRelation => fr.inputFiles diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 1d6290e027f3d..da9320ffb61c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -41,7 +41,7 @@ import org.apache.spark.util.{SerializableConfiguration, Utils} */ private[sql] object DataSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[execution.SparkPlan] = plan match { - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: CatalystScan, _)) => + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: CatalystScan, _, _)) => pruneFilterProjectRaw( l, projects, @@ -49,14 +49,14 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { (requestedColumns, allPredicates, _) => toCatalystRDD(l, requestedColumns, t.buildScan(requestedColumns, allPredicates))) :: Nil - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedFilteredScan, _)) => + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedFilteredScan, _, _)) => pruneFilterProject( l, projects, filters, (a, f) => toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray, f))) :: Nil - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedScan, _)) => + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedScan, _, _)) => pruneFilterProject( l, projects, @@ -64,7 +64,7 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { (a, _) => toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray))) :: Nil // Scanning partitioned HadoopFsRelation - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: HadoopFsRelation, _)) + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: HadoopFsRelation, _, _)) if t.partitionSpec.partitionColumns.nonEmpty => // We divide the filter expressions into 3 parts val partitionColumns = AttributeSet( @@ -118,7 +118,7 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { ).getOrElse(scan) :: Nil // Scanning non-partitioned HadoopFsRelation - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: HadoopFsRelation, _)) => + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: HadoopFsRelation, _, _)) => // See buildPartitionedTableScan for the reason that we need to create a shard // broadcast HadoopConf. val sharedHadoopConf = SparkHadoopUtil.get.conf @@ -130,16 +130,16 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { filters, (a, f) => t.buildInternalScan(a.map(_.name).toArray, f, t.paths, confBroadcast)) :: Nil - case l @ LogicalRelation(baseRelation: TableScan, _) => + case l @ LogicalRelation(baseRelation: TableScan, _, _) => execution.PhysicalRDD.createFromDataSource( l.output, toCatalystRDD(l, baseRelation.buildScan()), baseRelation) :: Nil - case i @ logical.InsertIntoTable(l @ LogicalRelation(t: InsertableRelation, _), + case i @ logical.InsertIntoTable(l @ LogicalRelation(t: InsertableRelation, _, _), part, query, overwrite, false) if part.isEmpty => execution.ExecutedCommand(InsertIntoDataSource(l, query, overwrite)) :: Nil case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: HadoopFsRelation, _), part, query, overwrite, false) => + l @ LogicalRelation(t: HadoopFsRelation, _, _), part, query, overwrite, false) => val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append execution.ExecutedCommand(InsertIntoHadoopFsRelation(t, query, mode)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index 219dae88e515d..fa97f3d7199ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.datasources +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} @@ -30,7 +31,8 @@ import org.apache.spark.sql.sources.BaseRelation */ case class LogicalRelation( relation: BaseRelation, - expectedOutputAttributes: Option[Seq[Attribute]] = None) + expectedOutputAttributes: Option[Seq[Attribute]] = None, + metastoreTableIdentifier: Option[TableIdentifier] = None) extends LeafNode with MultiInstanceRelation { override val output: Seq[AttributeReference] = { @@ -49,7 +51,7 @@ case class LogicalRelation( // Logical Relations are distinct if they have different output for the sake of transformations. override def equals(other: Any): Boolean = other match { - case l @ LogicalRelation(otherRelation, _) => relation == otherRelation && output == l.output + case l @ LogicalRelation(otherRelation, _, _) => relation == otherRelation && output == l.output case _ => false } @@ -58,7 +60,7 @@ case class LogicalRelation( } override def sameResult(otherPlan: LogicalPlan): Boolean = otherPlan match { - case LogicalRelation(otherRelation, _) => relation == otherRelation + case LogicalRelation(otherRelation, _, _) => relation == otherRelation case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index 7754edc803d10..991a5d5aef2db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -44,9 +44,9 @@ import org.apache.spark.{Logging, Partition => SparkPartition, SparkException} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{RDD, SqlNewHadoopPartition, SqlNewHadoopRDD} import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.{InternalRow, SqlParser, TableIdentifier} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.LegacyTypeStringParser -import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.{PartitionSpec, _} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.{SerializableConfiguration, Utils} @@ -147,12 +147,6 @@ private[sql] class ParquetRelation( .get(ParquetRelation.METASTORE_SCHEMA) .map(DataType.fromJson(_).asInstanceOf[StructType]) - // If this relation is converted from a Hive metastore table, this method returns the name of the - // original Hive metastore table. - private[sql] def metastoreTableName: Option[TableIdentifier] = { - parameters.get(ParquetRelation.METASTORE_TABLE_NAME).map(SqlParser.parseTableIdentifier) - } - private lazy val metadataCache: MetadataCache = { val meta = new MetadataCache meta.refresh() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 1c773e69275db..dd3e66d8a9434 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -61,7 +61,7 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { // We are inserting into an InsertableRelation or HadoopFsRelation. case i @ InsertIntoTable( - l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _), _, child, _, _) => { + l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _), _, child, _, _) => // First, make sure the data to be inserted have the same number of fields with the // schema of the relation. if (l.output.size != child.output.size) { @@ -70,7 +70,6 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { s"statement generates the same number of columns as its schema.") } castAndRenameChildOutput(i, l.output, child) - } } /** If necessary, cast data types and rename fields to the expected types and names. */ @@ -108,14 +107,15 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => def apply(plan: LogicalPlan): Unit = { plan.foreach { case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: InsertableRelation, _), partition, query, overwrite, ifNotExists) => + l @ LogicalRelation(t: InsertableRelation, _, _), + partition, query, overwrite, ifNotExists) => // Right now, we do not support insert into a data source table with partition specs. if (partition.nonEmpty) { failAnalysis(s"Insert into a partition is not allowed because $l is not partitioned.") } else { // Get all input data source relations of the query. val srcRelations = query.collect { - case LogicalRelation(src: BaseRelation, _) => src + case LogicalRelation(src: BaseRelation, _, _) => src } if (srcRelations.contains(t)) { failAnalysis( @@ -126,7 +126,7 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => } case logical.InsertIntoTable( - LogicalRelation(r: HadoopFsRelation, _), part, query, overwrite, _) => + LogicalRelation(r: HadoopFsRelation, _, _), part, query, overwrite, _) => // We need to make sure the partition columns specified by users do match partition // columns of the relation. val existingPartitionColumns = r.partitionColumns.fieldNames.toSet @@ -145,7 +145,7 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => // Get all input data source relations of the query. val srcRelations = query.collect { - case LogicalRelation(src: BaseRelation, _) => src + case LogicalRelation(src: BaseRelation, _, _) => src } if (srcRelations.contains(r)) { failAnalysis( @@ -173,10 +173,10 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => EliminateSubQueries(catalog.lookupRelation(c.tableIdent)) match { // Only do the check if the table is a data source table // (the relation is a BaseRelation). - case l @ LogicalRelation(dest: BaseRelation, _) => + case l @ LogicalRelation(dest: BaseRelation, _, _) => // Get all input data source relations of the query. val srcRelations = c.child.collect { - case LogicalRelation(src: BaseRelation, _) => src + case LogicalRelation(src: BaseRelation, _, _) => src } if (srcRelations.contains(dest)) { failAnalysis( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 587aa5fd30d2d..97c5313f0feff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -59,7 +59,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex var maybeRelation: Option[ParquetRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(relation: ParquetRelation, _)) => + case PhysicalOperation(_, filters, LogicalRelation(relation: ParquetRelation, _, _)) => maybeRelation = Some(relation) filters }.flatten.reduceLeftOption(_ && _) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 0feb945fbb79a..3d1677bed4770 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -563,7 +563,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha (1 to 10).map(i => (i, i.toString)).toDF("a", "b").write.parquet(dir.getCanonicalPath) val queryExecution = sqlContext.read.parquet(dir.getCanonicalPath).queryExecution queryExecution.analyzed.collectFirst { - case LogicalRelation(relation: ParquetRelation, _) => + case LogicalRelation(relation: ParquetRelation, _, _) => assert(relation.partitionSpec === PartitionSpec.emptySpec) }.getOrElse { fail(s"Expecting a ParquetRelation2, but got:\n$queryExecution") 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 index 398b8a1a661c6..7196b6dc13394 100644 --- 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 @@ -317,7 +317,7 @@ class FilteredScanSuite extends DataSourceTest with SharedSQLContext with Predic val table = caseInsensitiveContext.table("oneToTenFiltered") val relation = table.queryExecution.logical.collectFirst { - case LogicalRelation(r, _) => r + case LogicalRelation(r, _, _) => r }.get assert( 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 67228f3f3c9c9..daaa5a5709bdc 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 @@ -184,7 +184,9 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive table.properties("spark.sql.sources.provider"), options) - LogicalRelation(resolvedRelation.relation) + LogicalRelation( + resolvedRelation.relation, + metastoreTableIdentifier = Some(TableIdentifier(in.name, Some(in.database)))) } } @@ -447,7 +449,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive partitionSpecInMetastore: Option[PartitionSpec]): Option[LogicalRelation] = { cachedDataSourceTables.getIfPresent(tableIdentifier) match { case null => None // Cache miss - case logical @ LogicalRelation(parquetRelation: ParquetRelation, _) => + case logical @ LogicalRelation(parquetRelation: ParquetRelation, _, _) => // If we have the same paths, same schema, and same partition spec, // we will use the cached Parquet Relation. val useCached = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala index 61e3f183bb42d..e83b4bffff857 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala @@ -19,14 +19,14 @@ package org.apache.spark.sql.hive import java.util.concurrent.atomic.AtomicLong -import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.Logging +import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, NamedExpression, SortOrder} import org.apache.spark.sql.catalyst.optimizer.ProjectCollapsing import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor} import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation /** * A builder class used to convert a resolved logical plan into a SQL query string. Note that this @@ -135,13 +135,9 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Loggi rightSQL <- toSQL(right) } yield s"$leftSQL UNION ALL $rightSQL" - // ParquetRelation converted from Hive metastore table - case Subquery(alias, LogicalRelation(r: ParquetRelation, _)) => - // There seems to be a bug related to `ParquetConversions` analysis rule. The problem is - // that, the metastore database name and table name are not always propagated to converted - // `ParquetRelation` instances via data source options. Here we use subquery alias as a - // workaround. - Some(s"`$alias`") + // Persisted data source relation + case Subquery(alias, LogicalRelation(_, _, Some(TableIdentifier(table, Some(database))))) => + Some(s"`$database`.`$table`") case Subquery(alias, child) => toSQL(child).map(childSQL => s"($childSQL) AS $alias") 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 612f01cda88ba..07a352873d087 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 @@ -216,7 +216,7 @@ case class CreateMetastoreDataSourceAsSelect( sqlContext, Some(query.schema.asNullable), partitionColumns, provider, optionsWithPath) val createdRelation = LogicalRelation(resolved.relation) EliminateSubQueries(sqlContext.catalog.lookupRelation(tableIdent)) match { - case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _) => + case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _) => if (l.relation != createdRelation.relation) { val errorDescription = s"Cannot append to table $tableName because the resolved relation does not " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala index 2ee8150fb80d5..0604d9f47c5da 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala @@ -146,4 +146,14 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { ignore("distinct and non-distinct aggregation") { checkHiveQl("SELECT a, COUNT(DISTINCT b), COUNT(DISTINCT c), SUM(d) FROM t2 GROUP BY a") } + + test("persisted data source relations") { + Seq("orc", "json", "parquet").foreach { format => + val tableName = s"${format}_t0" + withTable(tableName) { + sqlContext.range(10).write.format(format).saveAsTable(tableName) + checkHiveQl(s"SELECT id FROM $tableName") + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 202851ae1366e..253f13c598520 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -571,7 +571,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv Row(3) :: Row(4) :: Nil) table("test_parquet_ctas").queryExecution.optimizedPlan match { - case LogicalRelation(p: ParquetRelation, _) => // OK + case LogicalRelation(p: ParquetRelation, _, _) => // OK case _ => fail(s"test_parquet_ctas should have be converted to ${classOf[ParquetRelation]}") } 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 593fac2c32817..f6c687aab7a1b 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 @@ -268,7 +268,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { def checkRelation(tableName: String, isDataSourceParquet: Boolean): Unit = { val relation = EliminateSubQueries(catalog.lookupRelation(TableIdentifier(tableName))) relation match { - case LogicalRelation(r: ParquetRelation, _) => + case LogicalRelation(r: ParquetRelation, _, _) => if (!isDataSourceParquet) { fail( s"${classOf[MetastoreRelation].getCanonicalName} is expected, but found " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala index 5afc7e77ab775..c94e73c4aa300 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala @@ -42,7 +42,7 @@ class OrcFilterSuite extends QueryTest with OrcTest { var maybeRelation: Option[OrcRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: OrcRelation, _)) => + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: OrcRelation, _, _)) => maybeRelation = Some(orcRelation) filters }.flatten.reduceLeftOption(_ && _) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 2ceb836681901..ed544c638058c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -282,7 +282,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { ) table("test_parquet_ctas").queryExecution.optimizedPlan match { - case LogicalRelation(_: ParquetRelation, _) => // OK + case LogicalRelation(_: ParquetRelation, _, _) => // OK case _ => fail( "test_parquet_ctas should be converted to " + s"${classOf[ParquetRelation].getCanonicalName }") @@ -369,7 +369,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { assertResult(2) { analyzed.collect { - case r @ LogicalRelation(_: ParquetRelation, _) => r + case r @ LogicalRelation(_: ParquetRelation, _, _) => r }.size } } @@ -378,7 +378,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { def collectParquetRelation(df: DataFrame): ParquetRelation = { val plan = df.queryExecution.analyzed plan.collectFirst { - case LogicalRelation(r: ParquetRelation, _) => r + case LogicalRelation(r: ParquetRelation, _, _) => r }.getOrElse { fail(s"Expecting a ParquetRelation2, but got:\n$plan") } @@ -428,7 +428,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { // Converted test_parquet should be cached. catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) match { case null => fail("Converted test_parquet should be cached in the cache.") - case logical @ LogicalRelation(parquetRelation: ParquetRelation, _) => // OK + case logical @ LogicalRelation(parquetRelation: ParquetRelation, _, _) => // OK case other => fail( "The cached test_parquet should be a Parquet Relation. " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala index efbf9988ddc13..3f9ecf6965e1d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala @@ -500,7 +500,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } val actualPaths = df.queryExecution.analyzed.collectFirst { - case LogicalRelation(relation: HadoopFsRelation, _) => + case LogicalRelation(relation: HadoopFsRelation, _, _) => relation.paths.toSet }.getOrElse { fail("Expect an FSBasedRelation, but none could be found") From 4f60651cbec1b4c9cc2e6d832ace77e89a233f3a Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 12 Jan 2016 14:27:05 -0800 Subject: [PATCH 085/110] [SPARK-12652][PYSPARK] Upgrade Py4J to 0.9.1 - [x] Upgrade Py4J to 0.9.1 - [x] SPARK-12657: Revert SPARK-12617 - [x] SPARK-12658: Revert SPARK-12511 - Still keep the change that only reading checkpoint once. This is a manual change and worth to take a look carefully. https://github.com/zsxwing/spark/commit/bfd4b5c040eb29394c3132af3c670b1a7272457c - [x] Verify no leak any more after reverting our workarounds Author: Shixiong Zhu Closes #10692 from zsxwing/py4j-0.9.1. --- LICENSE | 2 +- bin/pyspark | 2 +- bin/pyspark2.cmd | 2 +- core/pom.xml | 2 +- .../apache/spark/api/python/PythonUtils.scala | 2 +- dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- python/docs/Makefile | 2 +- python/lib/py4j-0.9-src.zip | Bin 44846 -> 0 bytes python/lib/py4j-0.9.1-src.zip | Bin 0 -> 47035 bytes python/pyspark/streaming/context.py | 89 +----------------- python/pyspark/streaming/util.py | 3 +- sbin/spark-config.sh | 2 +- .../streaming/api/python/PythonDStream.scala | 10 -- .../org/apache/spark/deploy/yarn/Client.scala | 4 +- .../spark/deploy/yarn/YarnClusterSuite.scala | 4 +- 18 files changed, 20 insertions(+), 112 deletions(-) delete mode 100644 python/lib/py4j-0.9-src.zip create mode 100644 python/lib/py4j-0.9.1-src.zip diff --git a/LICENSE b/LICENSE index a2f75b817ab37..9c944ac610afe 100644 --- a/LICENSE +++ b/LICENSE @@ -264,7 +264,7 @@ The text of each license is also included at licenses/LICENSE-[project].txt. (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.9 - http://py4j.sourceforge.net/) + (The New BSD License) Py4J (net.sf.py4j:py4j:0.9.1 - http://py4j.sourceforge.net/) (Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/) (BSD licence) sbt and sbt-launch-lib.bash (BSD 3 Clause) d3.min.js (https://github.com/mbostock/d3/blob/master/LICENSE) diff --git a/bin/pyspark b/bin/pyspark index 5eaa17d3c2016..2ac4a8be250d6 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -67,7 +67,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.9-src.zip:$PYTHONPATH" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.9.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 a97d884f0bf39..51d6d15f66c69 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -30,7 +30,7 @@ if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( ) set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH% -set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.9-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.9.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py diff --git a/core/pom.xml b/core/pom.xml index 34ecb19654f1a..3bec5debc2968 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -350,7 +350,7 @@ net.sf.py4j py4j - 0.9 + 0.9.1 org.apache.spark 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 2d97cd9a9a208..bda872746c8b8 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 @@ -32,7 +32,7 @@ private[spark] object PythonUtils { val pythonPath = new ArrayBuffer[String] for (sparkHome <- sys.env.get("SPARK_HOME")) { pythonPath += Seq(sparkHome, "python", "lib", "pyspark.zip").mkString(File.separator) - pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.9-src.zip").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.9.1-src.zip").mkString(File.separator) } pythonPath ++= SparkContext.jarOfObject(this) pythonPath.mkString(File.pathSeparator) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index cd3ff293502ae..53034a25d46ab 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -160,7 +160,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.jar +py4j-0.9.1.jar pyrolite-4.9.jar quasiquotes_2.10-2.0.0-M8.jar reflectasm-1.07-shaded.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 0985089ccea61..a23e260641aeb 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -151,7 +151,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.jar +py4j-0.9.1.jar pyrolite-4.9.jar quasiquotes_2.10-2.0.0-M8.jar reflectasm-1.07-shaded.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 50f062601c02b..6bedbed1e3355 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -152,7 +152,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.jar +py4j-0.9.1.jar pyrolite-4.9.jar quasiquotes_2.10-2.0.0-M8.jar reflectasm-1.07-shaded.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 2b6ca983ad65e..7bfad57b4a4a6 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -158,7 +158,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.jar +py4j-0.9.1.jar pyrolite-4.9.jar quasiquotes_2.10-2.0.0-M8.jar reflectasm-1.07-shaded.jar diff --git a/python/docs/Makefile b/python/docs/Makefile index 4cec74f057fbe..b6d24d8599cf7 100644 --- a/python/docs/Makefile +++ b/python/docs/Makefile @@ -7,7 +7,7 @@ SPHINXBUILD = sphinx-build PAPER = BUILDDIR = _build -export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.9-src.zip) +export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.9.1-src.zip) # User-friendly check for sphinx-build ifeq ($(shell which $(SPHINXBUILD) >/dev/null 2>&1; echo $$?), 1) diff --git a/python/lib/py4j-0.9-src.zip b/python/lib/py4j-0.9-src.zip deleted file mode 100644 index dace2d0fe3b0bd01d24c07e7747aa68980896598..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 44846 zcma&NbC7RMmnB@bZQZiXTej``m2KO$ZR3`0+tw}Hw!i1?nVyb!V!r6kh?8;t$h~q$ zWUP~CuUx4h4GIPW^shER_a^oqFaNs(2f_og_h7YTP*s5j0)-1{R1H+PP<3&K1p)>; z1qA~7k51v=!j<6ljnV!I|JQ@~pW&`1j!qVKw)FNM{|D$_VE-48p5Ff+r>956OGLp) z&qV?AFYtdsLH$Q=H}z?rs_;NSApc~E^`GFD2CfErMt0WLCPvQxOycz4%yOk;W4FPH z@_nt>@6;=yy(-c1v4>=hF=o2e`IyKWz6Fl4B~)U=twtlwrzp2i^W3{Lbt_t|i$@{1 zL8|~NYU60inT^etOk}AWKoqwgZ_|N+T?fW zO(QMj{-f3*IAh)FXXfAyDRL6Yj1s$KTe46Ci4o?7VJuxsV-e4XHESH1Hs#(eowgx| z{F=LJ9qEBGlgR;FRwT9_nP~6g8Qw`C<{Zqu?n837_qq|o%kv9&4rouH;yvgy&1_V= z#E{{%?}0kWQGGGh)V+0op?M3nT6;JN21BZ*Zk;O5V6tfPP{^1EiC1)-Noo}gGF4>k zK5cVaDBCIHYj!mZ!ZYW=B-vWQXy|?~O@f@r?Bg0lnl~j3DcwLU2KlN|T$9evV?F;k zhP3YMuY%B9J7Dh8eT>%=o%_PEeu#R!TG;-F?Am3DOOVE#f<~Lx!{h11Xe+AwU(%%f8XNv?kckCb{U?=}`_9Jv@W! ze0GMI7kRpf_rDj7`Zh6t=ZHz)7F&*kbJE4scr<`?9`949>PJz^rO}X<5rM%=re95~ zg{4bw!^ z7m!N2g3OnuHK}r&a0OFWVaWyZ!9`hD1Pe_0=!0Ah+8&dT391c%sR?HG8%oxS z)MSYgf)bSUO$4OJEw+Uh3Pc*O7=h2u7c+4xe(+|J)=Nx3-YNer?qzE=5gYIgM#d@e zCSD`H*pI6ra~~1I#vNrmAkB>!_XLUJ&+2ai0@h(U_|3#8jr;3>8FQ5}_t%wW;hN|& zw&ybzoRptgBz3XGdLq_S6-po#bV*%Y#jP0!7X(!7D{vNn`ZK3$g27|F7BVh2IrTfM zk_NMsJt@#sG6->Qzn{#e9M~T5BzcnR49I-%`Ab#TgLPZx6eN(deoR0CJaBJ1CU-0z zR*0ds$tiuI3#@Z?)5xGqUVau-x-jNb2TU`l8mfv46e_OUm*((2F?D@Z$k zk))H$E zYWaH~FqbgWGko8x@SvLdEoX{$`c(Y4c-2KJ&9%a>KzNS`R)Q?s%3H+&5QG=S{11OJ#Fa#l$28(zc>Z(6$U!T~;KdO<}Le5Y8 zrVcnr+NTsC=5wLok?oq3`@!^PsspuNRZ`&-&!X6(L5)%Kg!A@{0{~R>Ah(-Adg4c| zXX2OyBJ7ndTCdiZ+^;ELV-jEml8L%X6CgKM5WzC>q5yPIjCHBR`mI(mXucfwLa-*c z%BFb!+pdI6Ps>NkH#X*Sddx6=fvxp${Bm=J;TQ{Air+h9WyJXt6{T@q)wzINc zXjK+}dpE1$vp1RoTFV7pD{G8L*ZOOSJUEtYBJ5Rh*$i}SJmcUUSRlyDTUN+AU>jp) zn;*pm_iZG0-K_8kR)FQBTG#gaH;`9?3HmZTn;`7K%tUH;Z74DBVBy13FZ`jwb>b__ zm~&xdf_SL;(7;njjRH)3en)}Rx>OxAX+qEd(4CU1Iv=s-leofgMkCtKR-)x8C$*37Na**iDBJSOC&1N@nTt6z zza!vgmFq@4sfbcmSKi;}PU-xI6KBX5D%xCFaQVL45{IeMhWBU?Dz^Y_b)P_2*3lgS zouj^dBEWUvM4CpgLam^reZ{9QW8dDiQ~?59)hH4{CoX~|3)R-RL^1_i&{{%!0Cm;F zwZ8oKQGrRO(vE5JVB>ryd8uL9VCCPeG?N_MCDNg4d8$BkDfeGRlaodKg$EpFX5uHk zq9@{JpwFN3@>Xfrdj3#@t-ZFzt=P87ikkw#V8mFBA7+26EABq)Tmw-~&z4+#ZIcQ5 z=EOpxiY60E4lBBl+u4E%(|oA(q3CGmepdS`JiSpVGi-*l^M1scBPdLHz%B+6&p0hP z4iM$#V=e8FJQUDNZZdA~Ha4vF?Zc{uh0Byr(|` z=v#j>)QR|F89^ms`Te1OP=Uk~q@uOfpw;lCc^H;uYrfi2Nb}=`BJj0>szJ7jVvFZ9 z;aod!Gm%yifYwlDC8r}TLuQB5<|Y3HV@^~n5F!qh-Of_DUy-0=l^B`!cquW$AHM&d z-_nMz^BJ2B!(fT_dUg*2*REm)r!=B&kwGmiCfu&lnt_@R-mV`J*f^|$y~^A^N>{EQ zw^Z$KjejWdU}>=gCy+2g=r&JNfCyw5g?hcN)0xRe%RauF7Q8>Q;Km`qji1fWO~lK) zYfrtTe5?Ebts|Xtt~>eG>hQuD{_FZ&63+{qC%^4zt@!#EV@o}RY8u1PlkLy$=nHXY zgh4ViK%AxPO}9qYK)R5{MgHaeg+-MUp*bJxCPpoB0FfLks9E1SJ4`2L9L-=>rH)7) zeIC^%DJZE~OAxp_bLuV;eROZpqUt&_|IfhHV)B`k{GUWK7Dfg|GY@a{vfso8#%*?f zOadJ9FWi|Q6BP1@^j!d1nbnsCuakw1Cv07ybHa-B{zoM7c1}iS{O3&lYbD)uHN_KuRbcWnQh^$jCR(z)%O9Q-NgaV+D9B$xLF|_s+QVmov-VKhs(ye6S`MT z(afw!#cm1_Wg4$Od4>7QbzmC+d zN(6jfKWBIIZY|F@!^>7I;pxdQ4QJ7wcWNlIx4Da#HPMeGQ5+P8d5s|I0~*-*9m*pA zhL#aM5f6M#OumylxE*a66?JJPw=wxosLo!%Z6&SgvZittVD06@i}6V%_7PKx(l(k{ zoDDwER1cG}d~ z%@X?X(t!W)#nUu)J)17^y4CDuO#-H0-QCS7LhX`{lz!{mRs%b6z53X5ZB{6KDLHN3 zI=`{7OB_*;s=J2n0yik}?=-O!yqbu`V1XCIGiFT-5SBcqm|gfqx>ZO}uWK3Qc? zE7;9AWTkc#Sc#F|khV1~T7tCE#SBq{sged2a?(vNH8F9Hssm8OR?D>ZZQT28rw`s* zR(nA|nC(_s`Dt`hc1m6ZTHKjTDI<*@wyopa?M6`+?w7jMtk~(&hOp=EJWUDt8Gg^z zM71AedPtO1`v7aU7xYr^7ysDNpUiF9^jeaaNzeprm)^&QOf_?*D_9&r2JDp;3oWip zS%+6~-Rb&qc<~vHIx#866?63R(w?kSD^@y2!zN=-9`sZ6fM1U(#Zf_k|BxzS@vgF} z>D!1M%Q{^Q*R)ICgkr#Dr_D{{)z6Caik|jnUn^M!YOS@^*7CD@o^C-rifVsq1a^{8 zDhknlfd+=#AIwk+QR9l4dB0IsHeAb&4HZ8dCskgp8@%j|j5b4~t~EIqe9#XvNCcAC zXyiER7&&;WD?u0p?FkO?oDQWWqf>oZHu1%~a@+XX65R@*^fji;J#3H*5mF$`L4GHkWJL>Px=fbTigs;6bb=NbS5?JGUv% zfh^-_zvg1KW#d8gX2#JWh}qWJ_HzC>p=YnS_51fSO*qosL~(MCv)A5?hznN`bOrDE~{K2E|E49N$Ka#x-tiC2>`CbxF=J0()y1 zef7y+(itU!P|xx(9cR|#UPF|`h3UpqE_-(_ixg^bt42U$>OLWDt3_#WEf{#SMsw z$Rz8E$SD=3Ccj2{o_mGY35b80x{#8j*!VChmB;>86-#+U;?)_#Uz(XJ61X(Yj;r)bAe~5Ny)VFz4{*+kf;ym39U0 zqo0W_Rob~(0dO;~A}yfC$_=SZxWa$tPCP@zL2cqq%1Ac3q0Q~YnP&f9$^S9@%<()) zHf}8eYfQ9&_vL%9KzS>Hm2tA+``wL^4Y;&&&`!NqM0{|BAcg^*WUhicgo`Wwc^gB6 zXC?{UCBF`ewvj}slt`2?(zMo>HMbn}yt*vSIsd%cOG4w6h_MtOgFD6_N~tQGgRRKm zg~3fb5Vzi;5`FMFE-Va!0|BU)7g`LTDwV0I-^IUK%=GKac;6PB09X0^COzlqC;WF> z=;d}<$0Gs(but10A^j&U>>cf#?fy|O{ySpeSpB0~+;{qd%YO`Sw+Sskbi(eq0*@*x zyWww`@%ZFxk2T3`SUxd6N}SWZ@7RSFQz|7c+B#sO$%@QGu?%`hw%BM-^8N~A9Y^Jd zmQ1#ej8A~yfMA-Gn0X1L91HStFTM+9sGlkeBic?NF~V)&(`r08q7fac_cx74eifIT z;X9FELEA%1@|JhnG}*A(g`}MsXO|q09i}bWOA<~XNQ~A*o_rY(ePx9UP>rjK%HC{z zln`JJc||((LRoMgmCzBfN;NHRH01;Dr;UG?jYJwMN-Z$EIwm2J9!*>=vZ?d z2b~Vj8`7*ZpbEzy5`t%)Y)4WbXvONmlo_3Rqd!Y9Z2uMbX&zM_ z;l-j6tJ?`NmBc-#A2^B1T_2FuiG^TV!{N=9esHem*S{}66yx;hjke@SG5N-0#}4k$ zb+%07*7hj23YE37V$Yq?H(<_vXZmn9mBX-;!PsL%mrAsV^t^>x)Y|s!miOTP^uE&J zst0#HB8~W+3UwyFp^*b;3(yG@5RPIzNuN9jFj-mdEa|*Wy)pGBoD&du8Of1!ET z9FOrN*jQP1=5~cdZEk`vUF@u^uT*0n^X2Tok6vD2%QfIzpfqU#FLOA#Hr6mGZDC8F zg}hk;f$FqRJ9j|bkH%PbGS_Ym%J3|1VQa@#*D>12gb(#Yu%$+TRCsl-D@TC;+wGFZ zP_}AG212cTv(nw|*v^Z(;mM1y`tg!qTEas}Tb7dR+INPpu*dGLJ>GjXkvo11>DvcC zWIdTPTbBK9#<{m;e@c(>>MY{-9zjb)dgz;Ac9cso3(QFNh&?}B3iNgcS9jiZf6DO9 za@SXM{Armlvu*F`#pA!jQXg)Z8}Ms1lUxcbsOhh0L6{_(Amw<~9+0&{B5lsBxFv(w z^=n6c^Dtc-#4fR$@RB7yk%0g3WarV-)bLhO59&UG`-aAL_?<&2|0oa}Pj4 zzs4h*^{92A96&Km9Nd=sHdGzy<@}9t#@xX$8{jFc{70^f(St3hGdlIl)F|}`jAvSi zA0B`b3w?1|(fFsf4rifpx?;o>Gn0j-Fl|5&pM`A?h$i9rHdQeexzxuk%rZM`x84S9 zK&8;*7qluDik0O;gVc0Y`Zo>7Yy-m4hECBW@|ilangXp$w)MbhH?jU8y(n=jRQ^DP zi}Hw&L+>^Ft-` zx4IyJ@P^bapUK?FRRI@*@V55-6P6Z7O@(7#gc=e~jR-SFyaULzw$^-t`xoeKganmR z9!FLo!@!ENLVUgj0+|P)1l0gY-EZ)p9AbETHjFz+2p(@~@$shMJ7NO<~5Xmf1p?@6Ai z4U$&I`SAD>C<7T2nOA2%Kxhtaz+dK>cp5kacT9yIHOb?+KO`8x(%=bUUh^I1*RsTt z0$(bLWlT^84cqko?jmPd016+FI|gDKfGm4jg84V~4Zn7cAx|H9Jj*;nuasIh*zEuA z^%bIHS)Td_VR)3PprxP>V9 z$00L)v9DAT`{FIfmN{9BF1Tdj!z7wLmmzf{wZeRR9=CHyn*hwfA5IBVz#DEC0eqs~`NeWTl7}>e{ignk1 zH<){OSd>4+sG*yHt#q2uXCQ}YR^kdE=F@wX&5;!-i3su4m@%cKJ3v#f;?zY zi;a)Yg8j2Xh#jkd3RN=obBG1(2pxjt8J2#Ti?#rj>ET%w9rrE(#V9bgFNKdlFjygS zmB>HVZl62L$8$Jz)SKULUGfC7&^Fbx;D1!wG#Go7OKZxjpQ)bhCBhN!CXsl{#ek6# z5Gn~*PqT_ZR}%7{@}LWMOeuSHIByZ9MIh8N>AOX~3>OPM?62V)+gbGvk`$qw;6!`Z55n z!#zWxEs~pXQwp?H`y9OVnDG!9|6BexC9`(*`%k%UT%k>_()8eL=6X#|z+2@0&v41| z#jYz`6Wp8H0Of|F^!O{+p))& z0R+7KLoX>v!*|`(jt3saqdOG2GE_{+#{TuQfnMPcpCxz@#!H%#T~lmgAePd^SL2lH zsUD)vMq+Bntg7(!MrXf_!A>x!DX)_4q@)HWkFu11nM9zjHXu{0HHAGjUX+*CfJZ2a z8p?L*oV+na)-m`46yZ{wQ!uxnM8af@%o&Xs1yc8TY&CZ!25-;FO|(-7z|D@l>mp6im4DMu zVFf(FZaUZheD4RdTp10#LB0qgegC%j5&mk(q~jRNKQ)Ro2}M^XAfPb^;-yt{ zIaQgh>N|aG>^j9 z21~PzGsm}jGZLFBo%+G`C?NAk+WMjSzQ4SP8`F){9p6s=$YS>E?VCjLT6m&Lg}vTDHYemuY@@-vW7?si#gpoZ*l6tKjI5q3;c%)02;$ zm7mq)?gFOi52mHs5R@^lNJHa7gVW8RTZq-^6Mj-bk$jRY?mIkF1X9;O3gg4FXNtcX zFou`4!>`JgMfgiuZ12q4`Kly+J()kog6KOG9PRIyF~->};=M%L$SA~R`p!!Z&NvW| zW(cx=@Jo$flpq+v?;z@iLDZ_M#QAreFh+F3a-?5kkVJlyLXXC@8jY^Lq?)D;J)ojV z9-?sZa0aTZuq4HJBEg{RbF*;5z|(D+?LKS)A7uxl-hFwJzj5M>f*ej2L$m9ofDKVq z#=P`Ncz=>JpM=!9Wi5yC6^tsPs*XX&p;k3nO?A9p$@DzaLfljQOX#k9gl;hz>aLgD zd8;63?a}NB(-&bBfp+`%$;)1_xYMH@w==xd!wE#s*^HsKXX9~2+-u|g+s(b{w1J7{ zghex_2ke&}cN^Y2mfW}Ltiml3qB+)Q%sl=@J38HcG3%gHv~ndu`0B|+E#FHlZgt>i zw;?rPabDnC19+Zs=v;Bpu*CN_ObmF*gx{n*^8snC1BS0hWX0N-5m|J6pjRbCoukP^ zD6YB!3!fBFwFeZq2YDs;XCYH%O@#b`k~F?Ubq9^_gW9oWZK|@G7*u@j76r>3?6%i; zjVb036S>BjEqrxBl7tar$7Tf{Aj~ab`v$t zQh0qwtEJ=;k%pQ-g%aL=bwLXHQD}BE3Gpa>nGqJ_Z$zJ!2X1WG*4@lA`XoByp9QrW z=sOA;1SFZ{-*dshTC0jGNWIia_KSPYj4F2wZco2tHNNAE^M=OJrtR`*G2A`5RQdDT zKr~HnE2(#H%rGYR=*!cY7L%Beu<^dX6T zW zqK2 zP?XVNW`C*m1OD%(hkpzT|EK+*%3e>;!q&oBPw&5}?En9fVEZ}a>x^Ur78xA zTR4wt?|rzXyD4Qz&$E32ir=nsX1crY_tRG2F4ErJnd9;SS-MWq`bTBk%(14dk^;OD z9y56ouatXplB^#>%FxdChj@9hIr)x7S3~*@<)U{2LThz`y z^eYsGj2h+hk^0UNhDo+I8ob>`(wI(CFAKK_!Vq;p=*J5>U_N$0e=nrw=1a~9@Lk`T zFMgi_^1a^ekV6xmuI*MW{e-doIiTmfo`#AeLj~n1s%*BW=r_X;fp(bfWC8YBa5K*c zK6w%!jaW8Ykt545yp~@Wi%DLAJ5!EI$I5(SdkNx#BYcbB(Gxd9u=pA}a-LO7gSa;J zNjZNBBq#R^N)Ax*@N~VQh$o&8;VUATTwBK49kb7{4?RTB?!QSD4FV4<4=rnYoybkx z9WsIv_)FbjPtnZE%IbH=D`SWK%q%^*vBYI}zkVGXb_~Tis@KIQVW(YAD06$2_b*_TqD}3k#T-fN`Wu3nP|@R?{YF$M++qi|<#N8LFeR>)oWe5V_f*-RRAB!gv-kS1%_I zUd-s>Yq*anA;_co)b8jFBBFTFs*Y)?ahQ*8AdM&tiy)9W$6z86$bumQ7(v;5>+|bs zx=jz|k z1?pTV7Scfw9+4pN>^ySQ^NcM*jY*C1F$^w`XZTpD!FEbLx};*Pp_@t=!Hal%ywVrz zVL_kr-LLTv=P6n~T|t!vj?YKQa?sCHfu^xXM-jMaEQJvvw>r+uRC8*AN#8)2{u|F z^%6BygsBQ)!1IjwY$FKp6(UC{@CeDxn2b6QrT~gKP8ur3ike*p6s&xx9SE)(Z{7|Z z80xf(3^vyb4I6F;6M2m+{r8Syjv+1}#9R+9&llv#uu-ewcj}DjRLBBM_&$L49*eGm9zsBA+Fh;UWe`3C zLri4h)RMKfx29umFUFd=|Bh zLG$1{5)VQR$@KwZffYSw73DNVgV4JwVqrtf5-~;l>HEe>#%?&L-ASZx(1VNUrqraQ zB8(fi57XYs1#mRsO<3Xw;JET^J7;H$}L|4*6353>xXPya4+ zG%{l_IpSl;B~G=P=TwE=LZ(D}KFj%5vK9h{0{oFZfr{lU8Tdqpdh<(lKm((hA+lvv ze^u!4s6uJI6edq?v>Ys}Y2gRab`qf*GBUzn0R!9tP$ey#PB6HLAP+;wUPNwVDKq-m z$b7~Dkg?Hz&L$GfhJ($&wBe9ndtWO4<_#~o%jSEHj;Jwb*jzoGy$oIU4wy2UJAZ1A z(o1wuRLmUU5&VpCbWM{e1{Xx4&T1CIXV>B8d%%&;S|ibXr0ga&xlwC2#S>|3eNo%p zh@9-qK!NT3L)8%db|Do>>hp=4M1+Z_;1<|<*36mST8Gwb@~1=k15nTB6-hmtk*)T` zZ7p$-#70bqBAGOp(CTWwOMwTO(V>PL17*#_X)bNnXR9SI$Sh+4Ax937~qYKI_kh9`I6i zWKZjWq=#P7e4{8mu7h2j3`yN(r4mXJV}!j@O-BA4-1iOjpt2KuH=>UkS{*H^>qub3 zGImTX-XD1LHjaP$OJRYPFjn zdlA4jBz0I+>s^JYn|%Sv`0D9kXF9UU4Or{`Q*MAfhed@~h=z|2

    encyYo)zHI9u zo5qy0N6^f}S#Ew^R_97A|Km2K(8q2<^}T(u{5~dq*wBa_-mUZ2Q)Ae}9h2Klj4!Bu zzniMEr1I2IdWCQQqO}`YF_PQBk8s;LgtQqG+fp?mh5bye9SiBxOnz z9bKMvkI|;oLEc>5&fbUkr5kM7SR?D=`$1W5_i)aqVarp0Zsoibc=MTy9bf?1MI5|O=3eg>S<%=v|32{mi+1|q?WiQ)dPhh{JB zY7l*exDVsX{IOlACc}nWisdFl5xILGg5RRSnPdFA!4e%WQ)iLJJExS_#VfAh>Fg0! z2V~kdSOF98VW?rk{%JHg$}~4xC8JP6XFs_#4qcI$ByH-e&8L{2FE{T0u08vzuH{zc zo0^Fvl>_O;!!Q>ZC@au(uqNf#PTXuYZm-=q7IJF#&q!7C(PD1laDadkx3{*$03TJq z1hF!ZXi&ZirKgeuQVFTs1nSaV7^Lh1Mu}@#`5vug+!ozicF_3S_ffYXo6$N{Jt`a2 z34s55ebck)!{%Q9@rM^7t5Ojeml6B2rf0)wE_E4Vj{5KA`fe=AS5>}6&wHgC8Nvr| zVJE4maQd8Q{fE*9;^NKGiciGDicR>{Bjn2Ub`%Nm(HFg+-qIZu$I;<*O*tk1JqCSd zBs5R6iHsLtk?%Bx)c09 zmUBxCht?Qq$!DK41ksFSB`Ttu8egL;A>u?Er&}t2<58pE7mMFSZ2t3`41TfV?GWT@ zSh&nPKK@9Je|?q`uL`|D@uU>iOi8_&fW0<&YgS(omSM?H{R=cBr^9C#@b-6GR)6EH zChW%r_t$l1TD00>;QR6m%L_td1#I}bpy|V}nxNg}P3g}Q)wDG)uUq*aa4+kP7=m*b zQTH72h%`?QG?QZoc9;2f+{!xcOjV7ZF|s97(3>UcN$W~BG>^5Co&xJ6s?rHo6gTShP@WtM+~Ku& zB@1M%7X`7%(mB*C5bHb1i;CCpCa%fBkPG0cio^);Yyh;G6{&Pf`tu|QN_wJ05^|X< zZy|tsW1g8a)4Pc@sFP>tu~vq$TPX3Z*todo7$~C376hjgSQQ2&YbRP!`Iih}y2uq= zl;egktNVE0Lys#TQ|x z0Rj}^yQzQXk6Fme<=3XcP;v-nlX}k+4at)1N?V2%MhL9;@Jv#eVl>AjC0+Qkn1s8NBMq)iP-hu(iN4riZDLzT7D)rh9e*0oy;hbtqABsmcA2eLb*xfjB`X zO3`hDRh)XZah}K}9i}lo&;JU*bi`C8@_9xhUpyFZl2fW^Vcvw=0TVR_wrtR^q z$4@@Te3X#Q@AQ7z-*-PhN{Oc=t3lZYMo#dQMbcEte)QHv;ojb}sXoJ)$-~lQq2%u{ z8ZNnBxImYcAYE2|^nCVE8(--X^!5ri;pI!RX3a9b$NbWb0VO(3isC~!=EKl`&%>Dl zrkfdUmwhhK``a5n7>Jz*#Nd&>GuUj9li9x%OFX2iTSXjpHUxVb0XdBYlJ%))5u?E( zyhDIR*Ro=a4a0GuN@Z%)!paBoH6^mJ{%GKU_&)W}JhU$4!W^-A(91Fz)RQ z8CR#*Bt_?{tDOu|QjLBYI%ve+s;AVP92~X(yCaFc6XIpfj$pi%3rd@tpjcglP>#?2 zv1N?9xb?Md46&9ODPHgI zh-gU_*%TATzl{tHu`WJ0%2B11YA7?lXsvCNj8W6ECIQS9>1!XFJDK%>_Kn8ZQH8tmrU(QDl#9-D7=udO5WV7!1W`B{kCq(zPBN zHIOjko7fnVr77mL>Ta>52$cX{8OP~`&d_4^Vex#qo}$0EnPpjB;eHEqa(Cs<#mkmU zekisp@BP$`5?qpnW92Zt?y!chhJV~^{jq?04Yc8xuj6sKb=)QHowL3slimSD6(Kgq zfYMY(bGzz%Y2DqKkGI2KM`+WEZ_A_7^(j=7op7)Q62?%dH8il@)3;CO4yNKb>dT^& ze+D^M1hEaRIA*KQI5eoa)epE*vC~k)SG%uLfI1Du90}n*ZA<$`0tTtahV1V!>d zm1ud7tMh3K!?^+%-v(pHoAb2o?jaYNxCIg8M|NjX|x-`6W%{?720_Tpri?HZ_N)jPTnQN zG&Pm9Ge5klJ3v=SsYwVmvu%8?m+n6F17G=F2R2aRyM-ykBJY3E z40LsIJ*ic)@9WI9V+uFqiImaw zHod*`J&(PM6TEkJ=jpjV*YEWN+(dMZ;4C~Qj(leh2?1r+ZlQ5iDwA_E3|<4>2`~1= zQ|el!&qD634qls5Hnv{$_H#Ad8}ks?_6B& zY!KPp`nSiLRUYUcArdrU$l;!lh`LrXpU0Zaj_}a5BjG`Y(CnfeWX_lLGXpsKi12}x zQAum>;t-d?pg#shj)!?7EociMmSV<1c65fBjbU| zKrFQ(c!cd%!{~ok;thPJU9ZX3FQ6r71-3)CGUeY@}%cKFC#X%R);|l_D6$FI_yeF}|qLQsp zgRLIE z@IT(nU;nNP|Lc~Yo{@pIwV{ELm7a;afsMVj$$z5_rzpru4+MeA2nQ6+Z`!z})_ z&`4A;N31!rR)exKKt5ZNBo;x&ROT;11mUt=|rHUL7ogcj$&xaFJREMkC z2V5M2@GaI!>**{X77C2&Pse2ei&wp(_p zfQ#q>4}KH|t-WWxVwZ8}ugCMk&e)-tf$3@J z9P1-y>0wjs$^@N*>q60-?+Sx0XBs3ZtBV!O?)Ty?`kr@YZ1NH_IbAXNzYY@qOSzZX zM>o9qXLWF(fq-!TeTDu*xwkQ}|DUM#!)iNr|JZ1KXY~vQq^VUEvj(I^$>v4`BqY52 z%Pbo?5f3(#(&49ETGEF4nQCu6{55lcndoyaBD`W(6N15}0QMMN%>t)mPxJRp zl0*y^zNCMztXvgbhjKV2RayWf(hwYPwR-%hrC=6D0KSlN!eH38|5mHqn9)U-Rbd7G z13$`rVni5&;gmyKyh(e3{u`Y?ey;%~{8HHJFvR*=Aj=MnV*M!RZ!K)V+((lE(v({Y zkxHfRDbb?YpS4Se%W)!=Z6?l#ap`An*DEX~FLE=AO>z~_@ihGBb&jn8|JOAGHljx| zCKr&^vU;B1Qe<={^RMZ2i;}xIG2$Bz!Cg(p?h&O&fGe21meTTtS&&sdoBH2TTq&oa zQw;*-+1B3inwfEb-jNE+68}n}U`y(j-WHXR%R8E6oWaT!{MZ%z$R_BVKP%-o5;kN~ zRtF z^En@?0f+CFwO`n=Hf!YfhcWP!yNZlNG}!q&!gH<8)?o95)Uxhd{3)u(V^WLc!PPVm zAwSL@mE;5`I}?_Z z4gY~9+3&%~VLf|%^nkN2c*NG3>|awLDw;vOA=T~h0%iiglmKFqR9a!?P<n z^xSm1PW73MHdEW|@0g9bBg8mQ)@zc6-gIoHP=U>BeO93+qwoWYo-RptTpVoh11;VO zK~-D}8Z?gVn!Nx+`xce+uS`>r6L?RbI!QVi;lc?H7Trbpnw0ur3*a+ zq5g9*SsfYZANKt3%gg`k_{`M8*1+1r)5P(ALZD^!9jlFh2=t;yyos3wj+!-)kEAia z5+=y8)={PyY@MfF){r7~FZ8l{gZ7hr(Ja|XUrmT2>DKXa&9gP`3Ru;Y(#KIEk#d|N z_2BE9A0Xbouki|)VNv@kGoml$E^#dq09L94}3$4p3bk^N1n;u zD)i|w{d9Z%MAwbv15PR}c_tLNRU=6ybaXSVeB0c23WR<$x(6jM&aD{;Zs6GQPH^aX znSl(Y!zN5m5?#SUT-*`Z60Q-kiZXw*=?;%6XDds=VFQr-5>7s8%2?19W!JSn+8AxmR!BC z8k?!}p>V4OuiC7;Lz!L7v}t-w8moWuSMFNKy|c#20)cBfJ>bnv7nftUJip zDn0UCeiu}=cF`vRYbgUKW}DP~{UPr#HS^|aUaX$4BtDPKUWG|&XS=I^FKjZytW)d? z1$%~D-1|&l{j~IL`Lw1mp-PMcMkwiSt%8(VO?#|DP0_J`{;yw&PBGxnKrkR6(0}%> z|JO!kZ1F$h*_DR0-2ppN&xbnFOo6KF}ch5FJ#c^c8GCDwqKWG z5nU?+Im70xQd0|-KHfaNCNyHor3D5T{x&lpn&iuH2n@uXsLzxKV}UAP=jFcbc8eLz?aSbJ{MW#C`7C z7;+zn?Yi9shG2~8HQ*dxzTA8sZa(%A<*R(dGt#t=sZRCU7x3ptzKN}~V4B*N64TBy zlC8Jqzm?8&sSQ11$N1;=;()~KxLnK;H*`EynBPy=3bm+#fl_Nbci5Ri^ z@zMnz4cF3U9t3Fc2c8;8X{mqfz;b#CgNt$P5 zGMn-ZWK^!~x$&bbA*~+VA>mHYLVXV2{(YN>3v*u9E_h_$*VWb8lM&p6i^5 zxqFvOd2TJa*7K0T|Kn`&(q>|CQ)Z1N(1XEI6BcyaRvStpVg>JMF8LX-#|0G9NtlLCvp#UBI4nubkf`j4r@HldDsWNG5$<4HBc;%-mzy=?|t@sm7;`B%0bv_tAeaKU$amXrG#`Diho&|5) zCU1*36f-u#cgtaG!d_@7mK*X`H9IrX!-yho|9UFwgT;sI#OK!#-paMh>f2)dg^61& zvheEEW`p|c{F?p*JpzW9X~$zrY=gx>Lnf|)=g!`WgV$HYXsorIsn)QKr)X_k*X_mV z?Ac14ESE4z<-^yuUW-yH%>_RZC?bi*WuhNob5~IBFk~}!iEMlm;Leado52k>nru9C z#SxTigImgy1Lu0$sL5)Hmoc~lzd;T$e*WVR47sx}koEFQKbjk%MLCn|-U!yo;Bkhr z+5l2^I|x08RRJpYA~F#p*<{0kxdXTZohR@!oa9y;Xa8%2N& zEkt~TmIg$G4gHeY>_$ZUXwdk(*IA|i)VEfQb2U!@t*aK-8^6GVLMj<9jSsv7E=(9v$V^uEejnkN%$L7C$I4l~=vq9j*6>!RLe zfej!HtWi-*`)a;@H0vJ)4+i&-J`P%gtqiFVXU)jiEfa_DW{mOl1OA)w-j~N>^Xsv; zi!VrGNM4aQmVQ}B{sRR_C6=5L=eD#tYT>0;K$fXiGlts#yF3Z?pYi$skURZvyi>Z` zzUT=C0I){>fAnkr!aG*xj!yqczc!_zZM(sW?EO-!w}ke`8ALrsb22UTiReNDS-Z}~ z6B63LzSi(hLz#k-!Ous0l93{Ed7C-uikM#f3-0~xtxiP25xdL{4CthuzgWO_KQ@dm ztlaKL&!b1rK~^TL5{$>+tVl5oGAWbDkH_SYp<)RG14I^OUe4$I~rJ9_BpQ3%1++fc30sgfMm4P zw0#&muyi093VkD){axPI_^(UK53DU5FIsr#jCNa3m+y_08HPHOhng-EM7IZD- z)j*6OL?#K_KLnGw!LT*LJ)jn>br9K(J47Y_=r8}1T$MxK)rXNp6G_XY_fup8cGpmq z^yoe(Ik(6Q+p6nYL#kI{adYiGSgJs^B|F%}z(N~KjW`W;y+Fqt%DYmsKMt_as?`jp ztvugf1~a6YNG{I-wX~(NfT3u#;OSQlFNje#QhJWGsf|2NPE*R2gBhMB47( zSlTq8!u%-z5QlRYj{Gn}V)p_8*L}q8uT`Ab?G67)WyqY%B=B_Sg*UO@yV8n<@uf3e zu<9vHu`T{hSvrC{M%J%$gFZSPRlkm(Dl~``xANuyx!6(2?p*6i=qH3Eybf|;uj`~? z5{*7&zEIpDht?Hwr;~EA12_0Fl7jxeih6{2W*?8Q-ZniuXD1@*KlUncael2)jOdO= zxVVXqsM`MlOk6`!u>-XGrx85N5=YK1zZaM}5G>o|>u076keqycBXH4PE`e(i8wwV4 zch~J!z43x+?FIJ-`78WHcaCM!J_pdCl3ef;j_40id@;IB(x%ZfESp=$iip-u?WAn`T@wO{S@hMGC5jX$8_5r{0_d6pn&WX7);*Sefk>D!K59=>qKleqf6_x(!zMw>j%$ zRgRC=RQ7gb%7`=qH#JmevNp;f%AF}8UoVsgh-FiTMd$p!tA4a-QJmv~(s(T&hJ`q{ ze@5M37+M1|Pz(z{FD$&6Qz6$*pNY-&r_)H5uc~U&;V|5B1siqm>oHlMCkf2tsz0`# zvpTm3KUqLs4a1PmM!=FDW;GB|42_81U}nPPB07{dX(97qh8E}E65d7I^2X~L+FlQ~qo*q45<*I%XH~-pnEGLV- zk$vr9@wPg?ZrGqBZ*pkzaN=u3@QzLt@?Xs*wD{t|$D}({6-p=!l<(omy@aMQ3j~9y zayTWdU<{(j3!*=)vTNyIVPM}c4VH9gyvU~u%CqdeMOHK91g*=M8F0Q>2zAm-C9or3 z%n6loHee-pw!YIu^Rb+&^(k?;;aO0k!7)m7PgDK2lB+&%{9yird6g~7}uzwCu6t>7^6O1hwO2kgt5b0vA}R zqj`-IeS)8z9e~YP(eB>Oe3-Ddtd=Y_8=@YlxEiNP?`qkSZQ!dVGvTIEX>Akmvl~n{ zb#?P$1h{eIiC+2KVcChhwHLKBW;?dB&qLM%^cKlI zUH}Rmu|sfEWMdHMC3?C-W}zrP3_u5Fb{a`TZQxZ3UFPK(%r-i~zBlD)&NIUls+`!| zH#^6Fa6#VZ^rCw36+)Y1MBSskzF%i$wcg;a_?aVx#r0H*x}uAfvrM|+vC@KCI^qE~ z!iJf*Lpie!eVuE7#~U1Cj=AkhNzA>Z`0OBG-X;CQV=DV=J0v+W!_A#cqkvxOR;1Hz zwAiKJF0haoIvc3TH(hkmq4(cb-`lvFQLz4t!24U&aQ;~!|5ecR9UT5M;cc{%WNhZ& zP52L0wJbgXa)L2jLSc)JjtY(Pt|lJgcou0=$ascD*39D_*GVcqNE)``9m|luoRfOf zXuZ2$rlB6c<2%N~J1btpOC03qtTUD$fb|@C1CIkbI|0yeeO!iEPRT#;BmJnD!Moxa zrt9|tBp;W40;H%cVn`0J=8WoU$S48znC6MdYuUy$n4Hwc4;%N0+Pg4=;q%&i%VoYL z>mwD5{G26cd%;vJ#JTn!QNv(@`c)?n*YO<28LF^K=_3 zF=P1#X3Y!2j$@%jkF{0<4}QZg-Z8dhjn7uu%(84Qi5NdgAF3BvISG8A)YhWZ8G^~! z%1mku0QRV7mnj}G^9_3+k{O5AF#>x3UVMI11)SFn(ki&8D%1mNy97AMyY@srqd|qD zt7^18*a|wW2pQUVt07ubD93{X=;Y4~DoHF2hlqLfMaUT@BJYf7kP(@B^QiZ%LQ;J& zcUNr83U&SJux=Du;$do+Xrd~3{{fqDa+f6H?9@z=yhp#F^%D!=6kg(d!p`XStYWR( z3Ca7QL+AVs+?Yaf(CFTy<6)dF7`VXYuODWoaIWOdp znd1AZT60Fay4Q>0sUnOj4F#Yti9+c z#VW&d?TG2){^EPRcIKW*GgS%Az^H^`a_tbF1Q=x2$%Xu)tl%TVbt(BQj4spKoXVudod6T2l#k!8tqNqEvM|hZP?YZ%5~Hxn zBIXLm1i51Z3rvI-@PB{@fkA}Or3KlAtA>@;5Nw6AzdoMW}g+P>Jc zWe0`Ri@J%{6`NmPg0R*Hi-KoGknMBo+E#twm-%Ax7Uf1B{579cAh6~?Sf0H?`|>{R z;w3H{INks!wly z+f2nR+b&nYc}$ZPJR=8?6Ab-~V*~p&CgKG$j<2)Rni>nN>KP||>i&Tr^NKs%UHgqz z@*BNyQg`ihZeqiW@Dom8Nb*P5j=KcGkOU3IP(IH3Jqu5`PxhabEip`e!uemw75f)* z5&!eY{BMU+jQ@|Rl(bxj&0p}f`GgWg=6SbR?pF#e%w}HoMPL%k<-61a)nsrQcr(lsPvbW z!~JI*4cwiKbse1?%xz5n6EGF5G!c_YkG%DO(mDoK0ckNyT_LvCBu(W!r(#i(DR0bS z2PrgM@=SunTfQ@gcfkT8Fh7Ei`uxN(!aEYnL9IiJ&!S%bLglvFt=svWjNe)UL~tL1 z87aW0zBG(Dn$1)}fb9JnV){uWRp`|0R8d`jHgfUmi-32u5ZP1wm3V{yf@p>Ju^6(U zB}B?%(a#~diJ0+_WFO?1%u%(WKNU;`9l$q+8%?&M0_B(bO2Cq$(=}|?qT^-8Dw&oeVhI2p0@opUM`E369%#e-9^)O4-505i?^e%^0 zCg92wsoe^tXL%6q@Nv zq~LzT+gy!W!qfFO6`{M{nAuGn`9RodeoHv{L^e}@3)Pzf{3So80L5_8>fA}vJ#PL3 z_RmL2zvW>=2Mz#WM*skT|IcLlR~z%cCdL0jUAa>GYq-h^|6hiy?Yc&bC^3^f%>0+#(AM~~h*$LSLeT{oX2L%oF~}r|3H!|I9aS4a1@)r`TLULCBV5iMThnPs z)M9>WQ6i9{vL9w`46CP3K7HxzhMyek!&+JKaY7vHeUgZV87}R0c75tW+_v%IQAjg_ z%o6BmNW;@G%2TvA5jD`lMm1Vonjs@~FQu?2Z$ei|-n=e6*sR9xXGbnx*O` zvH&-G<8sRZdB*Bxk5_xh$&h3W_u{E~8iB|_#3iA{zYjN`)RXk&Y4_N55DlNPefzf& z(c1^zDGr?AxxwihHKiWs>|u?D0KO@In^0dCGQbK#8cV#A^~W-dAOH%?=j3Y=iqk;l zU^h%*|2!Y|a-nzOxQ`UB>EIQ+@pnE_BLQ|q?XeIOpAr43QP^)}C+*IozINu!_M?`g z><#AwiAjK0p^}oh`NLROu=)hp^uQlIUP_hk{89F?D!VHSGbSNrEkY%`C4Og^L`?8h zFEq!a9cuX6<%dlI?-yVXV+Mv?E!yCYMSN3OSsJMCWU2JQ5Yz_*q&(0LiztIA8%VCt zBXqOVBg!Ru`ZNdK#m%{vJvn;`@zwrwj*@XDJ$pAc=f;>_@RfS*%$h#U=fS9azrc2I zXVg%LOk!5aZ3c_3jaa`lELscivXrIRS?;#bQwzz-GXY5)f=^#;>m(Y_inxdnzVt|4R) z4^)p=Ge)?z5D}(59I$b(cd1KMc#6RJL80D1mOo(ph zURg!!TB;l!Rm0U_8E+=%mC%Q>BxNcByNu$1( zYFncLQ76UI60`^jbRWypN2}*H%+%$0bYBG_2psr`p1iG*-dJw^swjRoGR=OJbCq&} z#ZXn98kj!?v0>72++#kO$S`Z1S(SlKq8#&U-LpAiE^NR5j*`)Nlmd?f%bR)Cue7vK zf@40BTJ<5bOL1Zznvxx)^4q?A)$RZZ`bC}wAqKV zNZ;Ij)m!che;b9q({1Telak*I9e7P0-#ZVAM8FqGr7@QRUdI@=8s2jI8RQbAWh=H6KEBU*0Z9f3DS|8)N8x%d~;N)63*hXb9M5bAKN$>YS>uj$fInjIILk1NBsyW z+Nze5L1_mGway4tUt)16<4Tn)%10vPseVqp4d$A;Y8;>7DX>zRp%Eenv~uCx2^CNg zoNk|Izj9Zx0=Jr&JB8_6bJ8AUQD)} zs53(crv@AYUV`nG$;-$$?aAFSc-Z{;oq^shpv)bnxqQ z+epPO>7XfW=xFzzQSf2H z!|ff=WM;BlBfhD&$EwEvBgH9VoY^`U|;$ur&J zZoP9=nCCvn=G7*M2y2~z-^b(GsHW?9pi(&aD(ilR@=cUmaijSAmcnwe282%EeAKS4 zn0D27m%$2Vm^IW}Tj)p>J0;Bv#n;Bup1&AqyNrMFkin9T7 z)k!2nt76ch!5n`ptKy~JCDgJ{v<rZ%vZ9TX#iN^@NnuDLNElM*c-+#Rg{W>H~mm>Hi^UZ{rBikH(p3+2-*ifBS3G79 zuJY{h!y*L4o@bn)!k2|g4p5AH5CxGG2e%9Y)z!c2!^>bOb`e8>!gL06x8b6(p?TZV z;Og7h<~OR>PNG%5c&F#~%C)xgS72h%G}vEa|CLH#zm$ ze{Vsx>};k{77x2aCES$4M^TZ~k-jOrwPS1Hr~eeG&-*1t{KQx2iKO!vHCm;(fsK&^F*Y zYR`V59%mZ}7Jfx!C04SxjAEQok(A?dD`O7fw}p5a(j;~c@vGze2zB}KhfDSd`T#Be zQ)sK!M>|tVN;Zq03Hxw&>qXg=>X>uuvwg+a#i?1g#ap0Sx^eJ~)-QUxEMXhPTBEJi zO1f!rL8p!ZkzWe$9D8YPX8x~j)IrR`F9Ez3+4;en_B>Q@XLFylM3I!;!keZi5;}> zzv+DzFlEWD|2~f=sQ+Gu;As3GjgPieb|N$R{>pu7(Jvzf;ufg9*&&*yl2ZiE_!mkG zsT0}7k)2yREayL8a9Bt&t<7TRFS2E?*?YQg97CM7SJKS|BpF75c=JO}mp}?CZ^!L* z49(dTc6v1zT?8?XJllp?P}0yxjz*JHI3VgNt;@wB#2iSPgB!$B7)j$vQkYFlfZFI8 zsc6E72kd2XmJ{>^vn8%{OZX!+ku9&yLO)m!L|eLSrI6S=H}i+H+5*l%RL7*~g3F|= zac2O1qKvr%eDrD<5zCU~G-$`{3c*x~Crg^@H&MJM!;)^;rx^teccsmbq);{IU8dE$ zTE6l)r)J}P716@XK|?N8k1)nnzzwz2WBmipYBg^^yB#G5n;gQ3hvM8VGDZyh)L z@S7)6zxtUx1Y5NVVCrFeUiagy_Qw>HrUTy$K-S%~HV&&#NeCVH9q7I5+}oaG9OX70 zSHxD{^(>gygi|Jf^I5s+)n_h?q>@-RJ{k0jFgtyu4_ApgZ!1^0VJMGUoyzXsV!okm za$pb(YbON%#w~y%$Vjr0Y`T%>!u|S2EBvCvZS}3$fmDwQ>hu%P;8T;t01i$0(*Rlf zp>4os*Lp&t+v{wH(|ZE*gIpFUa|nDkpEN-eP0sth*F${eKbwl7=*G%OG;9j+M;ret z<(mu2$mrx}4v6D5geYfEZJLa8F);2H89rxIbmTrOli|V9|I=yY&UKG-EW&mVRQKr> zmNz?qN+?F0AZTN9|L&KfI>zI_(H|3QLwt6;002mkfPZgh{7=Q&zcAMSbZPj1&>dIa zmaZEj38$`Kzo;E&{uG6ZTuL2pYiG;57>COwsl_HA8aeXt;tR2o2od)JD$&uV{F)YMeejF62zn9d}Mv(SuRY#m-5U7Y+x$+~!SZQHYGPYKVf zh|9MWOp{qV=r*y5J3j@cHNkL8-ydFl(#m$zoVz5?Bp!G3mL1Puu)Gjj_}*Mg`zWbZl@Jhkd+KFc}GEY8?ueYuu>ZQ6Zc{ zGGw6653XVi*ht;r?(fK98zSzrr1YRn#YkH}z(Tg<1-k%MXs)zW{%TL+av1CG> z^1Cvmk_NA2Nk~~uz}G^#00Bbwr-HP#;D~eT+(KOyY3?*-x}umobZl*IO-$7-j;4u? zo+ym8;+BL5o%F;OpoG$g?tVjK(nYD3_lF?3T)Urg0_ZCngx6U|CsN-bk5 z`s`qB;6yj}yuBjy@kyxZMcJDwH_Rk~PFN+vQ*910#DQiVx}KJlZ8oCW}C z{E{}eHcxBLD1d))zugP~qrqZ{R#gq_wk-sFphz7c$chr)@G^hk1ME)gnQ7uMka}Zn zjo-*7A#us;8^y_h_Pl#AS?L@1QX#j*ASFzA?irBDL~xz@NvV5a2C?_@G6{gJV=rPX z2iT>q5MbbTyI0k1dEomA%$zs7!hjW|Mj(#!yySOp%5;tX%+}wv_P*)q%GO%l*y+Z| z!XLH(johBs)SpI2c8{)~=PBbBUpF`2_BOwfuGHMGok!5m8)MlX9xk4qAl2NjqZ!}t zM)%jRBh^x}wY9f1UoYF#SFKd`Ro{o>rSCqrwV~5bO*6NjT|Fn4bZuOmSeYGzyPnG| z@1M7owIMc9)3v^{Ot!+niNXb>bKytf*a_X7o*_Wyf~%ejp0E-&DG#_;S;?P8syR=% zfvrB{^av7a@cC??bUFY{J#o$E(E+0+fO>L3)B7_WIE$H~?6jM5NFMAikB=icy28&z zzEmMnUa4mt;3Wiy>_`j*38*~vA7RI6!}qhH8NVu(R6j2n|Cs3$K3YIa)FmA5KtVo& zBUw7%>#wQ5(NcaQv?VgckmCTvc_+Ur314xzu_j8$pRSK$?3 z_U!7P^y>?XU~>mK5z7EDZU1odva|t@LXcsK5#(A!w`k=o4lJW)-5{qM(1ESDMw94n zj>WzLMfu(8ud5qVe^IiyFl#!R-8=AcnNMl@M}&mrJoHf5YH6>ImnZW(?DNIl-S?Qj zOM@@B#J7=-iRg$JNke37Z6_mGyj0*2^7-?(%JXilE8}A|$x<*hw2d9etTL79$4TlV zCtapzLE5$5JQzBb^Wx)??y$&5QJ=^5-hyeEAlxoJNBJ$tfUh+>EPHsf-Lp(1-aTea z+jEJn-D3l7TxMmNpO&s(+qJC_%_|x1kQ_qmb7z5fHs?h65!UHr$4gYh>~K}u0sW~P zKm`Dq(3T?B3g=q}#fAO9%ct1Zpj4%jid2a+9A6N#jPiq=1)YWZtbci2*l-GlovW(5*_4Pr^ zx~G3q!7!y0A>z?r$*XaN@y2DBCtg!Z>jcw-BSGgJ6c2E0ILNQUo0SZvg3l^Ew(V;>InL#DMd#! z{IZwVlsN}llae?V4iC9WLI9+#V+(<^JCHb3BUSd^KlA1szPlQeu0+E7Yewd>GN8Dq z+-T+w%#yejD9p3F>SZ%X$Y**m(UTRXP{G9(&fdVG4pe&5buJWcDWt?|`$TC+o;nzD zM5%n|rhb^Cv~8tK`){;J^4tM{AhsZ=i3rBTr2IDvAY7U%T1{rm$C$i;7Rlt0kw?*L zdE^!2ga<_74f{}TrdOB&w5k|K1YVirXVjntRyx0TMMi*bW!SuP*#<+7@K=GDLZyV% zlV|q$Ii@Zs@W4k$`}zf)%USZi^YSWt59_o|nf#NeF|Zn#MR0)XIqALneu19nEL4RE z-1jhvNi({oy^~3gWUM`#L{Lo8PFv}$w4vi0dz)Qw`=1a`0h1$F-_rG+aZRNsx0kE< z3HM%PuoR}};YK+J*AsvX8XS*QARRdAy-XE2bg=gwNXBst3o_x0%b^_YMT0cozS27P ztYfJm{$Z~m?zb}A$~bK44RYaqTku)1C z=CrGk8p8RNY3}fA_uSK{+g)^Edig0lGY!a91#JTL=!W}EK5<+NNfMjOK`lsHh!4VW z!$99nc^fpEv^_fz?EEegJbLlw-)kzL84phMf4pf{Yz%7iBJ_~|g8456}X3N|claF;T9A}&a{k;;E9>_#o zO*33Oa9fbh&kh(gYm%xcXA|%>w)g@CJ~uRI`myql8gyc%a%C~&=P2k1Cc9ukH?RT( z;W6+zlwE$fr`!qu7B6+MGjdYpQ1Yy4vpzX|Oo2m0PYPlW;h9L-9(b>*XD3hV8{}hN znx+v?x=KK(Xq1)(@QWHZq{lAQutxpz4Gw(uKszzJ83o`h9J|ld6_r@KqAj%aaUUa+ zF>}Tyo#5n)(g|k0(X)-xp~fo0Bh=7PMcmy`|Z*a1F>8-v!;VK zZCPR0i^Z^2dll7q3k%ZECkL`8?8uR__EyCGTZ39e&VsZuGKiExIC#vw0cDNKiuaiF zK0!ek4+D)w(X$p+xWY_x5x<3CuX>3x-gOB-wFqa+MIv3A*Qvn{JWbl{N6tJTU|vCf zVtZDYPaT5#57gs$_Lj49SQQN67XA=x>-u2(8UH*eulTjUkJ8bP4GC6g2R9uas#|Pc z{6=OpPX@Fl>fP%)edK+Rqv;&oEK4@^INfRfdvYZNR7|M~z`DK9KZ zvMDhxW|0hon>#-mS6|<}(!14ckwrvv<=9neFQv}IoSnqvkAKzHO~$3Z!IQw4tLL6Q zo0$3NS6@b#Kaag?OC=J56C;j3@=gtrDK}C(Xb^+cwtJvg4+^&?>CEeJ#Gr!`6JRQ* zX!C@24udDX)Tw;PI}FihPGk%1F1PJrId$BtV7NIW>(EGer!C^-yhNBG&10VurMP$m zP2*Bl+LBRL?Rtb~BilfsqQpwXxpHS|_?2;84(Vdk$Vn9;qePvO7dwmo3iH=V#H)1n zB)8l?fcxk4*kvWDE?)FD_ApA~=lKM}841@?f(f)am}TgNAr-l_Y;IYs6XvP}M>%US zP=1JqLk3Q4jBfO_vL&_TElj$S>fhyLx1Nwdaw?84CtEqnW?&tWGNh?B6_dw-m~_lR zS~B8oa7tzHmigM6^t>98P@HU~70{^?X;!H+L6chTFqZ*V;7cb~bc55hJf~|G)JA`v zp{XEb?YyjB8m(|~SkL%z0AD4oRH$NaQBn?Ds{MK5AKo#^F+o*3y~#?ii2eiHHnjW; z7aUABG}A8*;UeX6M(BBu_@s6kTyw z_J5uTqFy!(6HWnxwtKnv*f<&u2H6N>5DJb~QEWI!VCZ_Y$=VPco66)r)Sjc)C^>e1 z*Z3KyUy}+?u#F~&W;?WQY9wYYDa#hHLl4(jNZW`x8dYjT`w1Z%NyLCkJc0>R*TGU+ z4M(zw6~lu*hhyrgOuaYARpB7IRb!nwNMziAk)bxr)*1XpPl;eYU$&S~_rvj2l`75| z@pDe)b1_i0Bulo@&D|s~hlCNiCdjfDwIhq)B7W{8F<4jSOwXMc2b& zS&O}w!Yy|`t~Jdgj#OS)kKCK|pzq^RMUF>L{I+Wq6W;G2yp^TIy?#R_ z|6x~7MCK@N5vC(X8UoeN8{)1-Ip7q5H>9C8FJHh5wh98g1KFKtc_;pLEtfRpkjrXH zC)g7C0&S<;#h*92Qn0RC$Yz?IV#Exqc=@=rQW)4AfaQ}1UcTY;+eAMkmybRff@@W6 zc2`x-CAtFQcPyJOQOIDf-mVJoauaUnT2I@mg-ZUMZ(Dj2VN{tX@cPZRQ8b8Np&b;=}-?j<6~CdX&<^P1s| z!X*ynDb*J%<&lcd4>uQ8*q*^R07P+2vyULDEF#7^Bq2FFJ`99PUA-kps&&<^)^rc% zz7?3NIq!4E1XsZFhVFpebSL$ACvrW*C>@r;=AH-^6*_xeJ|{XI4hSd0uUiZ(CjAk1Sc3k29bSi57AQLR4d{GKRL za~2Fvoy%tK!Q35%(oSHik4S54gqwFJ^t+B{dxDFspJIOzHYpsi&a@S69oIY+xe zhgsv+rse{wIFNx`MOy~DL8!h>B&1FjKJZB}a z1EBvvi!6sI<7_>Muxgli^vBk%)xNReC{Q8U_@Tg@?MAisC3drJ9vS>+)$HoAFMCrq z;U+124n#D3qf$8)4kpuvb-f8sqXmLsY#p}=Pp)1=D;33ak9yvYB72+9Hhbr|(-pmc zdi-X)R#=9I>dae*90}!<{ZR|;o`#M~5>TNntm>)WZ&F`$%U?*j3)A-q(zS$#?mf90 z6+G&rW*DdD^KOy99}4WG=&GrY?rk4Xx=&npsMJb%I>NGZdpxn;`HDd&aHY=y8fgsv z#yzG%u%T|f5yvPr+*y1C69F8zt5{rLxX`PYc6?l2B=0hAZ>#IzsGz;7Thci85_@NH z6mQf{3{Q|TgiMuJ0dpy6uiAJb3w_yUBoyae%*xLKu#t5rpJ9aLWslixbP{u8exYrI z9R&#m!{3cxN9znU;5eWJ!P;LvxeL`8$bqq}Vq+p4HNOvCmz^Q!ewl=#MmXfDkY43abD0tF|PJ^B`#L+Qe%y|KQ7(9 zrea9D0gR&zEKnoHZ36kmplJd<^{oWG<${miqe17p;>!O0_78UBBKp3>R8R z$hEkcX|j0d5$+n)!VFIx=HP4A{fd?204;m}Dw-D5S|x-di(*+;%lyncH;T~t3yApT z^LT_KI}IbQy z|77E-+${0UJ{X7F+BH8;8mr(;j)JKa79|u-aTYu#8Xe)RPuSU?m1yB;eJr|M9~t1` z`o5(>K@(5fN+*45YLIcPHd>*YPpC_t79tF203e$^-PlEtTWoUkqQpQ+zzA>H8}_gq z)}yOp!tFs);YjTOJVe18G+MgvO0XB0*S5=wtC@Ans=c=pauYo14dmz;#!bMJ!q$pb zku{#%9Gu3_Jj8W3&1WN4I$1?Jx1ln3slCd^hQjnt)4+K(Gm?+R*gn6(5x#tCj5Mu( zT8{w2twD_h|0Q%d-^m9Ly}wze@D@}xQq&{{1xvQFJu&@jKGTP1ks8TO_fPpiH(C*?t*oasVUSb5Gdx zOs3axR2U4DyjMY?2~X~?dl|;rQJNX{Q(m@^-jFe45nI5fs@_OTOD4oDP!9}U&+m&> zxN>UvMWQhi_)={94HB_a40zd$McaRjLwSq zSxXp|e9T@QH>q=t?8iqPKn`h}hpzqfTTVAT6LY&#oCp<60v=1uTMH)D^Kv;1^tf5I z4Kxns&11a75t{&(Eh^a!-=`&lX6O*{`8GRjE~HGUKePsiH+-*mC%&qhbDO{Fo45y( z&Fu=STQ4!Z(F;-k5<&~IpgEnPKdXa=O7`X zP*=)467vyQJG}3k@mEK#Zt$!+g?eb|AP#J>P89@8ejm4Q&^?5`fvPQr3wTt{MPup< z692_W*$!)eK7QCn6Qx&~5)G79h7YX9StA@~g%pcx)hxfD*9}fEznaoD4HZkAkaS^~ zAgj5ovux;JMp3D;0Ph$eBP?Q6H_>7*mnrH&QRq4-C`7AzdaS>;vV(gO2A9GFQ7;i8 z+2L>-oz08b-D&V0XD|eCoR@qW4;?t5k7cw+1wLg37H{n|obh4V|DgPw<{rVYfgE#X z&rFCOD|aK^5me2@K0KDdH@qV+FFBxMjte~8uP+w;3$BdfnLu^2**w`c*TkK{Rm4O% zf0ihL)Placp!YUvJy11PJY|<(ZQjj@eRxOgc1luSk@rxzM^UH;3 z6|o&b$WX;2HOJxM_0d6&bWh_7X>UKSZDJKU$vw<5_tK1d8EHN~-BN<3eyChi8U z5o1WiW{&-99UfvaY=Hv}*FYul1G{t*A*)uk&e7k7qJm_G(y+pjTagT3JWSNJlp(_Bod4BDp#1sFvJ%r*J6&nkcs0D?XAa)prU_ zb%Fa&ZD@D&UJeT$ZMIv~QrT2TLLE9n=&kz!(RMNcsaR&QX@u5feG3iF0b)#{8$25R z(YW~Pmrtk~_bN3F`!8f*-An|HYMOgZfBD7tuJ>PRv1nWSl zoLQv_@_zc`827b_@gdKjWg7IN)O62U+k)w=~NZNvi z3Z59i9V9vWJ9T4eT^a7WhM$LMr`t_E$wqjy%;!sq#GXXc8@)M%YnuC=Fc#HDu{ntH zQ`vZRt1rItoLo!bji?ffOb_k&F>Ri;wu6{mrxCNjUTMd)BY=;#JltQ`#2*en4v$;i z7Xoqp>A}mHUCLaZ-!nY1E>?qGPp_pCRjO9r3D^qHs8I?pvL$jvz_f@{y=j)5j}`Au z?pSV0q!m+cQ9XdXnzeneo@O*kO_6+-`futSo~JK`Rj(c7h+0-0cHpfW2$1rv_1AnV zAF^W|maR$8XO2;&hA!)0eH~jTbUSd%yPw@$=GWv#@LB$?Yu$t}P6Ly6HfwqisICk~ zxYFzprJcZc38*r+Lk-@ydSiWB;pxoGDp@VY@xdVTTdkZ~a4~HF;qU|Rj~@9#E9HPg zvV4I@M)3QyeY41l4s7{3o)Qq{db>W{A;pkwFshD}%6Q2d$VpvaC0y6ue;ICu#=i%A zsLZ6y0P2XhbS40uM^CzSiwKqT{lKn`o9@=|8V9`8w=D0)JRpprC0cL1E}}x8Em;|l z{BE&#h6_}XKfra$>3PHp4oS}ztfTcCF@B5o&*A2!xaQvgoNZ($E}i27+Uc6{3?E@<#}Fq34O%4N1lsB)GN9OL&doJR&Ai3a0joGFyJwOc^tEy9C@laE zAlp%$+X*P%+tSO+{7l84x;^DdnSI?u$~(L%2=B%toZg*s8iZUeG=>V=zm%JiiRCHN zLqPo!{w2p?80WzNnh+nd7!04%;ND>e&p*0UQ~%fN!WF+?E`yWuN53WU1tp7LaT|w; zGnJer{-SgqNV#x@)cKbW{fE>ma?i}ifdtHCJ@}# zF%nMOw$tj2p)opHiRA?b8I=cx;pH)l)>chfbn*j!Y zEKCU78|tR=Gx;;6qZc!)y@)n1cDlxL*I{XHmn!MG36Ucg>?a~RK82>r#1i+-VxFsm z!s{OtIQh3(xHO9bJ+~kuLum|nA_3pYjI{4E?;qx^PooC3ivf5!*BMKPz#5$pjC85G zYdt;PqY!^#J%_3?Li%D#nTBQplD%8E#Tu7`!rPEtEXWLaX&ow9;8W>$JpXK+W%Tl% zm!;FGOhw7wucbeir$C>GtXQd#g)r<1C8m?&h{W{2BG}H~FejNf6b}Q2b6g>CAbBeg zPD!x?uDAV2f^xrH@_-p(W#eM>r zyJz^~^a`j-LVF7lphr3=EFo(L#O3Wm)Vh4rgrpE}UZ{ld_VwkBG^t)T@;Bt_JxZ%s zj1Gidsu>3ZrY{!48jbJqqXJ=wVmfGkV3q_~Q{C1HH${;vJnLUYR4Pxvsk zfCI_x{N^RR7ws{7-I~>>*2+m1Y-SQY0Pb}`j)Hw7WjBh8arP^u2&3n`1_AQ-tETo2 z3F(fGyrXEVJqJUBG-*l?W2}uFx0fN&L&8rjLpgL)Ua%0Z8S6bzO!_sVsupr2MHCOd zNZC1H-xYcm3CgE}xBPKC{TOycFM&|y=s!lJ KIT9YYK$giZ6&Z#Gw;~_PO{a#W zfn+%A&&voYgGG@gBIg$OsoJ(JlLQav74_;H)%+EFVWHXau_;?O_vww=h~scnGEn%? zY>@b#HjfPGYuNeQl|m3@vt6aVnEJ`SxccgURCbn8ab$}Y1_Cq$hv1gr8eD_ByAvch zjXMMg?(W*SLlfLxf&~o(mteu&3GkTA+)*t5Rcb>Jdkc2K9`{5Gy|Kx(xZqz%D9It?K40QW6x2ja-Us_3K2h;zF^w_)3jW-s{lXdEjdDs5 z*=DW6KGN_YN{i&>$SfxO)gUX)jKoz2JT$Dy{%(hf3NN}CX9i9hEIH$XeK#A2Je3P$ zZQwUVWNv#5dp2hWja3LtIJwq>2u5M&+PbEfOcP+WFyp)>%AKn5O$wC*n>Qc39Od;w zW^=0Y$JX9LxAECz3e$`a*(m|$cq3bY)R%aUt{FvJ-zJ3J`p&{Xaw-GL>PoYQt9#EmBN!dwQ8BeBs<@$tdi)FHM6g- zJubQ%QjYnJM6=6?yaxg4M`lLKz1`3!+;nvdTN-qu`-1HqE({yDwGs{J56Nk$1KjR8 zBQiZPn^eWfKYp+!wGa{jkl$G-x3~5!q&30d9H%}ss_d5?k5BR{x*fQXNjJiwfrj)x zY3jW_&?6$W9My2?6XCFZ@TOynDMB`xTaL$10RoPVT3wnCGqX&fb#Ha#iE4OvUJ_^U zW)5qW{y1S;_c*&5-dZ`?Ix|8D#6rN}r)}i#h#$sIvKU)GCfH{XByblyrqDVm9>5-u zPKiYHdNvQlSq!TO0CK~^1T%?WcVGsy}SNvMPc&S!P7vg%361B206I8J@B1Jir} z7~QI5^eg^?@dN3?Nz=^T)%M-l>i4t4B!2hp(QSiN8_wF!{N*w)?6W@R&4K&2%{LZI zRIrC$doFcV%aC&4&yHBRu&W7-;f!FIvW$=kh5&DG(poOS{UMft14pbMt0JfsE~0=4 z@w~%0TW9O5CqMQ(H@4r8(9`7p0E!7lt$oh0#(aTlnX)D`KDs=Dvt>oP^HJbtMz0oA zj4Cs%=p*A@ueR(A<&cMc7;^LQ?)>i2w-*zkjpc`9A$J{2Riy(t=Anxg1vuD%?aJc zRl09Sys69KP_~br98%c*Jj)vi9Lb-%Ti@K0TspG=m(~P&YY&&`!QK1t#mSA&bU56M zk9TbVwuuoeQ=M0t9Um3nspwRt5p6@qyLQ+VcwT1Xlv+hfuhghUU3;Tcb*>>B&a+`w4fJ5{9%*CuDYvAyjyH6MWp%k5iSTAW-MI`28J=f-;!kYgPd^buQN`?Mu$F&gi;?l_lTk-V=-0!WMm6nclWHf(QFYe?#yNHWCkd zLuhNe2#Z&jhn4x7Q|T6~K+I?QOa5^-g}QcafaOQh zbqO4Df^2$Kp?PsN{1)EpP3SvWj%_5B`qdIhfZYd{eV^JF_fE-^=mPkFa2*o^nX~ngzYdpm|=iYgjSE}Q9**Yc9EMYo!z4!&4R<1V=Ze zxaieNa0Z-3qL4nYiE*6l=mxadbwRsktEJ-%%Wr?FC0Pliq#wvC>bR|-ZV!v}QlyRk z)Ur$EOCPwzLWqf*7(@^*0SV(iF93g%${o!o`8sc(tao5)ucXaqSDoftX`&{8JG~qb z%6i0|Be0_fVHOBN*TUL}C_-?CK3bWdef=6E~C9jttIM5vX}@f*e?HO+UEr>i4e~M0&=l7b}~LqHeSB@YVF)HP=73Y zWp`aZVgX0bTFiG}zuLT>;GeWC6AFerCAYE(=hFZL7k2OolditshG3Vf{&a;cyrAlk zSvgBH8p(S5(hpf1=2g&CzOO>d%R*L&oj7bY6F0;k+MNtC$?qAccmgHofR!GN8|;^K z@B-Q?av)BD#*s82rU5@Myk-!1tV5BP1J=K2ysEOp1B!R>2R=;&lLi|~kjKzl zSG|c_Pz!mnVV*VpnZUP!Re$RjWNELjld_G9jm6oSzA`zdmJmtT^IT-a&=8(fy+d@m z?fI&4nf#z;3}*fEO{Tce6yJhvxTeX!;Kq;X8v+45OnOebhi~>)EEZ>}8W0_Z(h4Ns zdkSR#(l5$EV(in0SSr8-5}g{HL`=3cpV~Tu(k@i9-wu93{&ZW2Q!4az-wv(CC>JQsR4zN6kEU#V$=QpGJBY243jzS;XK#`awV zo^pcIipD^}`S?p^{2ZW~d9%)gUQ%$7v>_m55`{3a1ZYT$#~h_N)5$F^XHl)X=Qr4C zaw~3@{q9xcHTO~AQUsza!_xIcW+)ryM&&wX0%(0-ck*i}2j&qb`n8RhhdI^}06AG{ zu;yM-ie=Tu30n7_1PyPK@SUF`Rnt$Dt;bkVy$33rZBCA&f<^>rwzPc?^Rfe@Y^nIK zMlDGkAag*H0WZFku;-jZ*SFm|pmv>}-t_vt0%(6iLC-F;A*r4aCBvz3nI(vuvvdier;$zhlZ9Y1M!fiVRY{U+3q zWPRhTgCs76zLqb0RVo*Mu@R(oi;^2z(lr4TQZ7f1Fo=KG#-PSdT-&bboD`YnelHcXoC`cF`r{q_e0fiD16P)o*#<6BtpWBz-v0;4dUwO_Jw3&Q-rJ}^mQ zBHq3zn~tKb$E)4ES2~iP7ib-wnA)!K2%~QNPRH*zsFc;WEi7P? zHXKVH64k#MrIpBH_@*5_<`rj;JICjrs4U=f(lsGF?q~!BKpECPQkOwlNFyBAJuSz6Q62LA0wrz+40<leo0+cT-rjAUmdjn}z^L8gs1FP{E7tJs}|5!rz{0c#j z6xlv}IXc?vh<_YGHFV7Gb%xBbMFOH!kyMH*v0iO7psyPeq52-virQ3;fqW|O-d+G> zlElA)1>Ga(Mq1AZwk^;^Dpqptg(w=gjg9^~=M*_2Yid*VT|U9Fkp~d5kv2(x(#sj! z^<{tSbtoK7CvGlvFOCZQ^M0UD937{$6qiD(8EiaX%@2W1^HabkC8xRY7duy{qu_f3 zXUb|FhU09}IF3z$*?!CPySz7sVd6JVSP~ z=i@~;{0%t$wAnL9D3Shnjlgj*wx;xTUuAf@w_ONXO<8~p*Dj4*7EXZ%tju8?KkEJqhFsRv zy6}i79g(`wbDqy$+hCIUpeAkD#K1&P9W!ga(p~azF>?uKNlp1zP9o!>y*xF>a4xEm zr`jpOQOTBW?83%|ZSp`mb6IQJ%>b(o;kV$p5NU3c!7>TwGwK1|)JpN#K$>}K>bDL( zPPEh0s7}D?I2E-D7f6Jamr=~mF*kIaO1Z00!%8-3y$uu<-mtZsm(oCf*cObO+hyrhJ+1z81@ySMifb-dLEOG+1}*@)kp@+*iKhrHY`no8cs zOO4x(^`RWu1}~Sjf3$aYMuh9Z4AAgHPj`_$4@PILi^Jbi{}3^2qv;Y*MOhF)Q$9p6 zhf)6M2(l{WF+Wve+hWvKR0q4$Q^*%s(nJ%12On+lx^H!{fhI5-f2#qw6rd8xl~-{eWKJHxu%)!UfF2JGknNZ*zcImFF{w6 zf`eRHt13)2#Tat?RYIyg0OFY>C`Tgv5g|y}^$P41QF7DrlaF-CIb7qF5;+Wf z-rK2Wz*+BH3am!&i9|NJBxqcmJ~{%jx`&GNSGb-;A}uun1ecCNq%2_r9vDhbxQHsh zv}E{BzKob*{Iny(gg(i@kHf$7(bN&IexSk4OzAV9t_RL(0{(_KHH_YC1rAUFuk7t4 z`@XVEInw3FG(-Dn%Q#k;y-k<2g{dGfI^$mQzt6B>I+6fDN!HVB8QfiB9q zRak5ml+eMu;+d%4*L|tcM{i%Lu+c$t%avjEFt4(;nusm<$qA-R3kD_&z{1XsqS#4_kO3V3O1Ox~ll*@t-h&1m1-W&U|>R zMAgQaZGzU+uz+Jj6(*Y*P1ucb0$kdt0pWQ%4Rr2C(t7OcxJX_3OKUf{Sw8Nyj+42~ zmUzfDZtLLvq75{YV}U?Oyg9Ou?vl}m#~)Ch)%EFhbGG4HQ+XJ%7v@yEN6M@B@RPA_ zQ65B@n4&I`+irtEAbE6|zDwZ4O4|DBV=Tnk&6a>$j9VRE8!4`oNQQmFmUuNZ9gUGI zaa>Ij5>SHF@J#eglePdsYTa?HPP-~@Yebvm=1l&xnWUX_8R^M#uK5s8P2=Jc{k0W& z^K9~v9#YTn8P4ACJH%^Td2|V?$9LV`r{lf7436iWAE-Z6roIXDUB&H}t8XBSEs%UJ zy(hl4TPrtd0oF;o9p-sgU&_1tPfM>YS}_*5NOzqh5r(K{ zt-2p#T2P5L2y!n%suq%#m?ails1v_;QqWrWnFTLe;hteR8B4b7tOf6mc#F2CU*%_# z+0b+_VP+BJu5(}(>_I@*XF)*Yz&ta@3>d30!-nsO%*GLe7Hm!j()80vXcBE%5Ttx> zqd12jfa!1<1-;ZG_^~2LYejMW-0jhdthoQXr9(oKU)mbnMXUosh*!QMYl@QV?s`;e zSrxg?I`gNQtoLxh(inO)iJeleh{8a!dQx;m#6|CMF*Rea=)g;smL`w+Knr<2rTZ^Q z?H78c;*_~V+3(%$-XF<2+qv=0@n3x8wX)s9@>%wta4igiI;9X+KwjVTIJ+?L|K7pq z);J(AZgo&XRe0AH@maW<5_VDDyJ-33;1CPxQ~v(`kcCWJQ<>9By&7PC!lbp+3G_^U z2?pr6C;SX8ce5iFeFzC-`ujeoR|7f?6(Dtl&GHQ(4B7G$m{OMbuu~pN*G!3Fbiw5U zi!TGI%Q=RE#7{7wPBRO=Ia~J;!cM9S0 zP6p&5)pjlTL`}wmFPG3tn%L#yJn`d4k7vs-PO4zu6w>94pF&~kypSPRK<1GJR^ZoW z4_wr@FnoXGEZWsIv@7@A1iQ+n;k-DxzcE|YuQ&n}deF8Ju>W8@2G7Ie$B?X#o9;dJ z@VJ4gm6fW**B&rnyD0aT1uS&#;Kg|4ko>C2#0V20uAy<;`leol)bTZs1(xCX$SHM` zI0^=0NcVE^y22_=Q53fF4@zs1EcZac^s9%^6tx^#XQoGI4#$jrVuJ9I*VRM0YvbGV z(l-8Aiq01n?%~A9euQZ|{z9Kh6rIT6a=C_H5TV4$D5ASax`w$^EH{1Jus24gL!#4J zuZK!NzSH%D;aLcCwF)!AVc!$r-;+};a09)%)W?aW=)!j4_O&L-I$pAwYk41OqHXYQ zhIFV$@Y733RY7Fm}^z%9PX>JsIKby?l@QV^DX?|TJBQ|R<={q1njMj%?;XZ_RWp1YeJ&lygpNjukrqK zxw^HVNAX3EzJT6AX8YWOJ#?;{pB-sEk@gG49P&bgrD@hZ!l*0HGIt%1T!Z8p_wQD2 z+Og!dg4Betiq|QYL$DtVdTnXZdKT<8&T|sWv30DlI+ zxs(wb^;=pFIXSfAoUR^oYcp}NCkap%1~hx<(jc0wpZlR}G%&P=jEu^UR-cY<%%8^X zxjNcGHkcA411Xa>XHH`~uZ+<7W8u%uW)qFVCE61%Fbceh&)={c;h^C7(fM=o#oytb zWioZhDVQX_M*+=lKRcvvCKhJV%RsDhezrmxFaCNvBPIuO=Jm_)#Z|xLvUqU=bQ1fc z?EzY@(51CmCB2tkSOr>+g}p;H6xc!IqvCH`lfQzXH&Nb`SxZDPs{NoGLK^|`v@L}- zbWV^c?H8@;qY6FldcLzq;K{zV$HT2L>AQO=6~kDf8^e^EO`7See3Xr<<|^k|IromB zO`ZS$=0r<@TmY? z$+}e_NA0Cib$?FxVIU7i))4Tx$vaRqu(_%)0^}(m^7w13Rdtil5Qe!j4}CwLp3npw zJB1LGh!Ky>Cs*H+0I96pDnGv>mZAX`lUvcOBY8+Ov#r1-iIENz@AB7dw>gQmN87N&rcV@tO7vA*nG6Vq&VaFJrsPLC=;=M zz5U_G?%rxzE(^ID0Vv~bKbel1H}zn14uyl6fz#(Wm+{oYcS;B2xiUUkN=e^s@-1Tr z9L*WQt!A!}D~$L2Pwh#RZ*u*=Jm#%Tph7^P{cHF7zmrz}udVCqi(&I(s9w{0OmB&S z*xGsIVQpv|uKE8s1l>iG&z>wcP6r9)P`a~bwqFk; z-oPd<)4x)ONlHV!Z)w%MUnd*r?G-%{!vqUdU{)*W50K;O+jf&<{2&GP!?3`zriw{+ zPv>dT$Ubl^XiTo7CeukEq~!5S&f-!GFyN*1LdE-Y$aF^`Tz%`kWQZj*1cT#8R#MHi zvpe1c+N+SHK1IkEPnHBul~`Jp+SNkoa!6^Lkr$%dl5@hCbkTW(BhbOUs$&wq`CzIF z+FBNmTdj@DW8-S~>(|4oiwcd$Y>lO|v1Hj3+BSCpTDz!D8SwHu>+hOSX9|Jpwj9~c zfCa&NG~lpE510+wTh+ZqKWy*Oc1R;fECnM2^z0c+hJlomTdmJv;(TZx7ly&`$*o(e zD{(!hD56h8lxiJR9i~)QdU$e;$&2*!N`Zt)PPha&Q$Dc~r={lww=)z@c1QjjGr+gW zKB+b|2?&;W*FQ5-@K3)qa}V0TNpG8egScb2jP6~l235&jO7$dq=eep4o7H06^^j#i?mnwyK zMeX4d_cU_bAS@2luxN}eUTeTc;`V~i|N><*^tMw+c&%=j=x85KQyJG7!I`6XYwMcWnNXReQTIn{mWAHQVp7p9mX6}pHc zqD5}qi;-Pvy>}1|d{#&*Znqs-i^Li#y;~kU4~Nd*uxeH=Og#K!ul&ef;dX0SWVD4M z;z0oh{@zenoN8KUFnt2C&yxLuoxd&=K4#qVATed_Q_WWqdw_pv&}pZG+kH$rPxL^R zo?g`Y!968+Qf(8kiz*wFD& z+~NPBE*$P5HTvjIGXMtxf&4^W_|eYkXYCDbtZd&o{y#MJGFmL7Q6M0i86hB0pJ-}5 zW?uc&w6(W!v@x`?{6AIyzMuZ3`q%)Ejm!Gz0Qpn(Nxs8BweH`mBK%d>KT3Tk_~Bow ze`a(2^BDL~cQuVwoVQ~_K#bTxKoCEveD_nqUkz0LV|n^drT!1(X-eXsWA@L1HqY@_ zTl`t-Kbqz5GZTLg^fc`6SCA*y6VT7vJc<7MJ;2j6v|jVYMuKl14iu$<7c1z zw~_AeQJ(q&{z{3|`ESZom%!g6JhfK)l>pZNZ^9F!wci6gb-wx)U}N|f;Aa#4>Vx%r n%1@t4epRN=`AhlVGu}VmBw=9xm<;~-Vle!83zshVy{ZZ%05BNuz_4O_7EM4^V=^Q-&57IZ!|A$0Opy0_OMWA44Y;9<63iU7G ze{BHqKj&JRz>4by1pt7<`hS3p{tR^rH^CbqnG2fIrM`svp_Ea6?A1sPwsL2=eu4E*OLd%w-%q zQ2Z-s7k)9pTM%g>NL6lW4@_2o=-#Y`!7yzFek}tL-_1CH?IGa&u;u}KoZBRii_n6u ztD9gg0Gh7k2Ak}{t}U?KN< z8>nxgMAR;J`Pm4@n!teZ4g$m+^FCf$-vElC7rzrMaoR(ff@#&73t@P?g&Ylqp4p9v zQ;L8_Mz8Zz&+DY6Dz_{m6L>??HevR{3xXI3JCE)3?RupGfxN+mK4yz6I0#rb;9ter z7$-!T36_Hy$*3Jx^GLt(gZ5Nl8mlTL%DRn3UnsFgksilJ#0A~ROXP_^H*IDNZxCZ6 zo1;;;649O>x&>FrSo*rT5qa@* zeOPiV!<@VrlPCsP_sif?D5H?X>j>WDoNGu#qa#Vw{-671_X?ZxwKSh)n3)5`k!u5> zEfBG!WH>KJa5&jtnmd5l0x$Xu$}^Y2rY0$`9Vh{CoYdO$il@)L#oXIsLU3%i-aEJO zj{JMOj?u&F>FB{&m840<%UYH2*XB}o*Z*zL>C^Ov^i7fnf8mYB>r(|=T6-M zv6x5buAa!Kh*U)vQr!69o1P{CeI(Q*6k{u3+AC>l5Z0oy=qqcTfz_< zWhA%KSJ41ss;t6yh(q?TlgTCrGjwnQh5|F@=APJPeC%{E?cvV|ZODYmS#PGToUu)) z#V0Xj_Q`DAE2RHE*`J?^ZHc*~lq)2iL>pM^VDi@ZAli%>@)rBFzI^>NG;Gv~yA8%M z6ZG)o);|_n+wfTvqE?9E5zH-A*`t|!tj@7eZuJcpeX^FG)5M6PtV}YE_PcRGC!~{K zWP32k903QLBg$zFm~9He#| zm#cFsse&tz0bc-W*(@WtkR7_NtEn6C6w;+=Qv9oI+hYRefdl|krj&Vk|52enh9# z_&zqWS9sSY{I0cm?BM*{<7yfjh12 zLVb2x>>h=mA5$OKFY_nP99F#++64%P-6q_BkU`^OV+SP)m79Kp+{wIv@0L84*^J^<>oZM?YMfv0rqBiO^e`aqoRRJgnp74N@`F(Xr2GdXHisFlvC<8TTEB zk@pyo1@}F+;6P8Ew4xsYGRrtRRY^Y^PE^M{N+1Dr(B`5=~%{p>u|$ zyvTbY_L**07QfEyO4!Fu0CA&e$t5u*N@r+|TgWI9;EVyx&wfe0W=E(~xA*LX4B)F1 z+UV(uOiI)9?D|H-AHVG<4D0|Vw9^0p9YdT4&O|+hZ#?VN*B#jAun%mqGP|c2rl}qJ zO;J1pIYwl!1!i2!HLcE!AJ77M$=MT|U1CZ_R-53Z{wb^9(}J3))%!d-9M2H5L2)xx z6b4z*<8I{AWf}dKKsl1t8oS6FL&v@Hxp;ai;bHxgZ9-TK4^Vg4#Tw$d8&N0eA2Xv2 zC6OYE!LF4i|BXWa!IzlwNT19(IJe<-OxuH`PyCYmza|_Of;W2u<$=+*$ z`q45Bfpg%#DA6X4i#z(G)C0Ooc18L{0}KDr0#)+mmNBtZgY2r*7aQKFK9E0%mq{s) zUmn{FU&2KgNxh|zW6&ZPloX~$``K*r9(Z~jHXR0&PD z-U7?YLHoT8?3*q$PZ>Qs(xy$tK#QSf0N#%OMyRhj!g5cptg|ifG0KOYE=q{+%r9P^ zg2d3kGO{EfdO!E74PItGn3NF0u8?lrf!5Im))B{w<^eSuIjeU6YwJD6zA~mQWLC)_ z4r>`E8FCstTuL((8h*jljiy>RCa>PJ4(0nkJ(%L0fH%JiNtNRUqi3mN$#hBGPs4eR zI|5b|2gDC_740%%}M#AD0uN7b6P+E06_e=Ihk478QNHSnL7QKch$A}cGRXA z!k<-rMjd=BbnRl&IUI^l{U6jjx9iX>{8kmq-97Bq#U4TgEam%F-SEd zs&f*7B{dZF)Gf?DpIXPS_ohwW?{w3u+X@}`tQ}Rdi6jLJ+VUu4FV}k91tm<=6qr^t~*erpKD~U zj(5SwYgRhx0h#tx&`7&U%UHf2AmH>Bq8PY7)?m36;odeh)$Vdb-ptiED#r8dBK%+d z7h-#Z?~xHYA^p93*0bZ60Y_GXP^C&W6arMzhV_7G7kI+56I)m)rZDb}ivecR!t_2I zfxFX?QdN*12*aG>z9yc>xD}xq)*XozhwbRFTz??*_`EaPx?r+%rh|57dq-Ip)==~W zf}pA?)V#r)wx)sU(R#B{Q;^rOFhALp`lC4zEJ(-c&@OYU4KtTXCVh52_px@^tHUb8 zeYpe}c%PO`QND18w&$5~$+~}WA;@Gd{LihO;*wqvenz7DAlvCSfX6JWm%VwwZV{Nu zSiz!wDzZOoP97WW0A__|X=iOm9w(r~Y7hd=OjPs*yAp(Wk<*$Lr>7VX*|nhH5ht!4 zwu_8_XauE_ygh}v(d7+9#@%g6Y`_NM;@3YX_f#bpOSHSSJ-~x34X|F9hi-$DcbtNv z6jP)kw=qPtNbN?D3mWXMZsM(g%ZSAt&tqF)St6w`Yr{7okoemVENX@Er@yXh@0=7q z&!$CYyUAjH6+j_>jD^K1R&^(`7hpzh*k0K9TO%P8=520LGY)-0{c-Qeb!)-v&>%l!7zr+`?=Q*ooG@L?gKh@s zX_Bk>gKGE#KXwS&1b*l_e9o`D?aXG$H7H|{kQ>VDaBP|R`#)^%|B|lThBPl#SO5SZ zF#rIJe|rZjLpMWxV|yDLQ)3rPdpqa<<|K4%?Ke4)es1&!ockoSS0y?>_YrK+#>}?6 zo)THYw?Waig-UF>)TpKT6y*-6U;1{Z?nI0AaLDC0X%v7(ZJo?GvN8Fx2}^j4>HRzg z$ktip>fqKk=c~|vhg0R+Hm|cy7gdQg=#H4AO?g6Zn`pofp0tiY8R}QRGKX$S5R-^z zl-MNOlZ6_IjL|QRVrg5Oinu>*SmH>vDfaJZwT;;2*F03~Ne-16O%GYKA~E$zMf(=d zaZUr#=b#?+9+P`~){W_3UtYO#fO`WKAAnz|XQMhKhK;8E4%LZ{8;U8X9&82*En2D6 zI>L$2=uQ%9Zl0}n;L&iLbyrbhxQ>&p6DI;SKXj;-jS6XZl@pVq+Ad?=tuXa{4_$X1o&nst7k8u-W2r1jp23qtSg0l7*K(B4jU9ty_> zz#4Gspa&ka>y{ZBbP|{U47+I_@v@yD_4&sZFwclPXuE)t_3T?SV|6}EzY$Nhito-2 z8_@_1rce}y08}V1`$g-~m}0(|=AvSzM>$sXau2QZ*&Cr>=IOyd@GKhlZ=v(#h)LfS zTaANq(8knyHUf2>98jeiL{Z44QInJrfWS(oUr(!rtl|p2DeNUYke}TUO$Ir&?gK*l z8vvS1{gvG#BZ!NRd%~F05$#%q8;F=H>M+39dB-;>Adz$frUvY67NM?gR^>9~45q5a zkPGC4iL$8(7MSui0J)(@Cc2F2^@0H-^~XGts#sz@5#zZUDUcGf zq&}|V&YYbS3?lXoFpEF^g+n#L@F`vk5gU_?>Vri|gGtJP1Yjx|h%k5HkIa@F$UfmD zS(54u(0t#;YjyXdO?&1PIDm^mOh5rFU|%{qS1b-jh>?xeVNkx2nzLpmDhK*)s<02T)uCp^(?ooHd~{ksNFDh;BliFD@>6TNPF^P;!A3OQnSH7zrV-EhsLu%@G94XrA zQ}I9I)t9N%Hwwgou$~btcv*H;cZ!2RaIcE_pPsjMz%QP^b$0jr#mg+ihZyd^v1|AkFSo&GGzq-3ggqR!>&%tW4!}=wSu| z+w0-D`S&04yxF!hB~%haj=feU}WX3E2N#!O|h~qPvU|Hwi0{p);M@8fbvmoYx@J6 zh%3Q({TW`(VD=#9B6WMV6lnL*uwki}{*a(L@l|C^xll4e+*EuhpeZEA0j9n@QJ^%g z)hCRaVAQ&(&Pml>PZ;w_oMBjhD~kU5JV3^wWh;khBFu#fe^^8M9Wi5YM9HC4b*{6o9+SYBa{{pzSiS7*O8ujB70jvil&@_G%Y6B+eFFt!6 z`|+Wn3=rU~L6QhMbrmdGsIkE&kSW-P)Dqg)RaZS)>(Bof6_{i!?VKhHHpyp{ml}}` zR_0-$o@D1Lkq%YMQw5++dH61xoGjumJY+XF7eDP2Jry?xe)*D@w@$m!_lFp2>$59v z!?a6Q+!6=|A;f6SR?H%Oqe5dm$Xn7qqp#rrZiAbw&j ztrWZ3chs!~hu`V66wXJqfyeS+0y8md0dN?~K?p@rk9T^tXbS|{OeJfP#h(V?+Iztj zy@Q>lD5R)1BZRC#-yf)K6B|R}b^QV9YGqb@pgYzzu=%2|7xBX|hDgHrQ%Uux0*)g{ zNn@iyqv1vUI3mm1a=opP_JwZ!gSqgaB0alApf^V^Ttif@P+S{uMr)98nv?SA)0UkSS+43i;s#hJU`^=f4er3;x|O9TJkY&W7HA{;mI(9S`2Km!*pWCQ4D8Q>Iu|Q=aF5Lf|6Ra1Oa<8r|uI`NB0*k zt8XInD+jL^lh38(D-+F`8R!+wJ$+EicnA$m+U@@^3b4<=a%Fx_kjul$(knFj=o+e6OiaLRED#U$Zb$}go|#M zboP~C17!e62Wm^>YS0IeUs_EC%AZQ|IbEgGs|gmy%eRh#M9&)0>7ahSt%40bim z#mAC`0VAFT7Lu$P6c&E8yY(#^3~kcYMOrGZ!z!q}W3Q3Muj4zOBsJrC&4Hk*NM5ti zChgsq8x*^*?bq0r+l;@rT(RaTh1GePEJ)eu{L;obStX*&qY|GGgqv(>{Oxs3Pkpvf zKhOp4DGqqiK4!nh&I(~ywbD-OdRs3%S~kI+(7SG)E}{2prT%mOc!!kUj$Z)tSH3C0 zl5BT!|F6^AkN8U&`OrLC31h-K@aOI6^X}{={%QA*HBIC+e6TO%5c?9++d5s>s61_5 zC}?r8U77*^Sb|$~+1c<$5PA8z?raW1Oq!NlnVNvw6d|uKjtP&Sv!4&k1}N72K)Hf& zAH4`NziII#d&YVkr)Y&m_l9*6#DVH~h@ox#O{8{pqVCttOLhp=AWmgoED`lOJS`?#G+PMcrD- z?Tr2tsvG)<Tajq2JC@$;^!v- zx3uTaoy~LN*(c-L%TSl|h{)xq;q-Fdn{?o3&(_(~3idOOS*e``)?y^LB<;$9JJF*&5RtQ>bgi`t7TdTwjTX<(}(Y^t9`(qO!liR{M33WyCtsztsabK z6p_Y{J2vqh_M=D(4@=!@)@-zC!;64U`Zr4O+oQ!Sk73YLf90sCdeLW?U?HsRHr_j-TWz4?qsof#G5irM>k zX-?Ov6sw$~p_4Hu4+qG5L2t&C;wZsDe@T=ucvo4}3~a?tWSuWZYC9xvL(yQe)8?je z8fL|LMbG-OZiAi_&bGmxM76&(0=tMP6oqKMfdj)G4reHYsIbM%ecmam z8gFFBhKpZJlB%xP4PW>FjTgP!d)y=w!Y4;N?J5EQ0E9?@04V=K)65NBOx+DV{~JZ~wRYbWOFVP`hN5wv z)m)G!b|dq?t)8LkY7`-tsveba>h8e9MlT%G-mmUr{&*>PmmofchClT>UioFTV$(r;#$ zbbT&LZ-(WOeK^|uqLc5Xzi>;LO*-^(QcfnySY9R?cb~!A$f|tWE#>CkJ#|gcpB7DO zX7|jZiM-5W%b0pR3Dvf+7kRjT9hF=>X|Z9UPg3PXb!D!wYFI2YlYanx;^Hiy;8tt} zWnryJY_v+!lBpaKN>;w|A&)OTU@O2@*AVQV^mL>?fd@W26r_DnlR@_Pq97a33mr$N z*ZZ2BnX&ta?k#tXQv-MtpBUZ8|KkMAj+5Qmdh-jVWRke%8C zlv4T8KWu7ExvA9keldnr=nhg%0)6Lz@VV;gMHxCJv3qW==))vcbtB4Z1HTc{P^c;F z{Wc;|Q+&Vy=!d%*0Oq2a^3ne&FWJS>J!E*)=w%#uIbBrN7^Eh3@86<{e9xR;tq-N` z#$I$*hCMxtG{36)Qs;%6CDMzkMS5#&AaQxoZotgb)0eFyIhaT=+*#5BkWO6D9?d6iIi}SsUxb*Qb z()Ujaa%leYyUm`i>3eACaef4p=cecD;J285?+?BA$MyR8J8AdFLEi6E8Gl9?zmA`? zS7J_1-p?q!TLispM9??8z1}sxPnvG@OkY^&D?FT-dD{B%{QcPzE!yvCP7{9b7hBic z%^3|hhwr1Ai-V_&mrzf5@0sKKf%E;IHL*o)@VXzCk;8JBdk3!NQ~@f=EcAtiNZ&dgkOmI$Wl$nKrOY`3M?S|3{O|k(Z6n!LaeQNg6ie$_9We-gfgk#CYFd2k6<9i=-S~r4HfKrw(cZa=I$+cQA4Wq7Z(5 z@6xw*a*vcU7ew~N#+VA6fVdwd*QF7wPPeoK!?XGTBzU@=uCv*iCw5F768rLbWc^r z2sieT<+^)NEbjZ05@do#P%d76v|je&+_7G3NS8xkVC)<~=2WT8KTl<(9vVB!lqYcI zHUQKsVqbf|Cf6+TZZu}O?>=E1&j4}A$WfhO-|ApT0m&QC-f}EojP-!E=Xai9WA#=* z8y8Sf<1eANZoXs6L4QSFz^M*aa@Cvbbd{oxvCp19;iDVp{#K_SGMq_(RRWYbZ7XB1 z8nJ_w`eo7wcxS(;UXN8(O5VGF7^^0Kj5e1$T0cEEmZLjhOk0*?4g~!Lb5JNfKmEdg zx#BdsIBkFET;8o+KOw6FW$kJZU7Q@RtToKZTkQ`8j=Nmhnp55D2_QvnIv{q*?{Wnpdgx)1Xc3^VVZlo?;1JNv{P&4S0+t=ACszI!>Mh79wgeF_W znJGhR|B>n0?E7A;_SW-PsOLG-x>>98^2XjBgu36jC)1vT^J72-r^C;2nS zR5H6(%FYg}06uUhKaC!KVjWHMt>8qD0tXKFkZ>pffWTe0@ZgufzNI!e(V3OTvvPVV zOcDBTEu2^ho!^5vu!Yf4(LJKP3D=Gk!Xc65%BD6StL$kGI46*aHyssd2?E37g?w-b zI$~+6*3i#2g#=o)boK@?V{*(AF4KX|E(E`8dw?|g%u)k0eIa3L3&0+sZ4F`0`=nV? z%n=3H&W#d1vzD|D)Yg{#|f+Xq)NoV zK7*&gK*_*$xw0Wkj3qV{NEt8fB9s9Sp;u28+;C1?Q51VSf>lt0!7Kloycc-4_GB0d zHAOLJt+LXALA2s;a>?&;S}FmcNHx8s?LFn3Oi694Q1=rVxK3xz%P7GA?HpW-4`^$rcca1Iw((sXew~EPTT|{-^f6X zOTD3T%V#2)E?=()v1KJGmehxe17@c0rY|5(yXbFVLUpIuEgPrwuewcmJ;8n<>GmPSVmaHnD?h+rzT)m3R$)Alv`!-R*CWsU#_3LJ2 z;6OB}L<4%$>010@@wXU6=1!Jocv7d`1D;B(FU9Br>j~2X*aw5+$WWKy?EqHltG7}U z@WDgkTC6)UVw9;x2%65#JmSa`piqDev*wvS2Kwa1E944&-LuKnH%2tlSW;AMgO@hU zK{~c_BzJNI+CL%!3p0+aL}JIRXmQ-ECEs zFp@poCFlmbeVJ@U7Cw>gPSXhc2h42%-CgI^=)V-4ATzmp4b0BsTR@7^_+(_iKS zZ2^p-qaOA#cStqsp?>kl>@w7KP9n5ph+86ayC}X)2h~kovZ0PlVqnBA}dsTO(PG? z>M|hK>wCsZi*O`LRB$=jHCEZ*4kABgaC-P;4}QEp=erNC@m296_Atj7lD-cw5vXxF z-(pPR%z=&lkF`w6#YJ)|q8zZ+1*uBe0)g^^+-$SaqC@o}=VdHuL?2wr=j5PQdcJBy zgb67*dR>fUseS)8go}Qs))dU|dgD^C25rcQ*8sZZfEtu4YqIjHc|&pXYwPxwESs@c zgPr5e4O_6aa~p=?IYzwWfbf`qdgG1ZI^TH9URm%1rTwA6fG*u?atRV(ruSYi+21>YXj`p z_p}$+1kXt!kuQY0YRf+Lk1R@#_t=S0=69V^;DGNfC`Y&P{x&3-dBHlt#X!Uk|@zS*M2Q6e6$TaZ;&S0mI{gRle#e9elFbK6`b@x-1 z7p{S}JOi;A#kt8^;!JnbYZ4#77ojv#&udu#)2Z-WC$^oeWu4Sicq0xNFuE#PI|!!} zYTOtBkQC!7IngNUu~8cOxytJ?=~oC+gizMVt^74vFDHc>+$3^aY6~YR&%3d+HI}#s zBEKy(s1{1Jt7y#u-L7;QQ?T%WS9F2aBXn!hRJ3eCZLhryLzLIUY9E%)=yLfz-i{zQ zy-0mUn!=9CRQb$psjrg?SZH|qrD{DW_oZ>QijM>8^cqN%7LhmirW!m3;chi`W1)uF zaJFY#3>Iu&lsiyq@MgRW{9Tt!_$Qb>i&rxGasv`+)MTycLzyoJKwfneMT}%V4!4O& zZK2{BE!)!d_-ID7n>i^mi8-ehHFt62gKn*?0Z0d$aF#uNztC+(V1Ir=cd15FDoxdp)o@DW#aJn2sNR!|LE@v?cst#S`2ITD@Qn| z`9R9(_G_nh=WOkT+_=V;i){-adowUPXv_gRk315|G1N-II6=2xkL4oH>YsafS<(~ z4W49vbhEs&JL?Kg%}j+=Z>M<`m)h~}sPn7S8|e&qyo6HwCb`K0PC`U3L0};8M7YK` z)TOG%eD6gs?NGjVn4LjPrFvRIH#EfjGwRn=v3_!~Fd(+^l0X-s`ad-!sr6LeMLv-m znGHX7J?*n~C1869Y@16e7r?6&cHY*uAVu(>q1>Iuw9~UnCl{xR@RxbQ8@gXT?ug|d zuV1GpyI=5Qs0Kj))!RZr^WsDq{Vm<|DSL6-FqdE%lR}Gy`?EI`u`{Hp>0@|tdm%3D z8nbhaqT2v4Xo4Z&Q&V>PTE0t9mx%U_g18cVv0c5T%4N$h|w4(wEmJ3$|+-3K96%F;F(#-C4U!BH?Io5#8|ZkOu>zynM{{rw(~aP-0~tl z*jUvRV20?3F%j;@%(_;>&ig_vZQ2sN{k`DJUw6T#NFC`0RH&1xv|c8%{)+DIxQ}(E zAE@(%=52n434i9jbP!AdS@#SaJ_A~3S@jJY&8N$hJ;Tni*$DxX0wOPhbqBiieqg?v zreU{JYjsnsoUqYjxjy;6c>YPiR?|}7$H%V8;cHL&)O?W3h?R9nZtu}@-CwFNuq(pl z%r}ifNE{ppdLgD-jJj+zN~;pZqAOX%KC5}uMtAQxn3=fcuNIZ#r9Su7qi7=e;&{?Q zf4H{cmQG;Jj2RQd*AwHqfFr5gS$(6HZ)d(XU#z>TLg;UIXT5Pa_NjnF(qrHtw?)VG z8&8w9c$1gn<}V``-PM07wn0*w%PgLE`y$;}$sgBZAgXX+D zqm_lS);FP$A~Hfis27i}{_&L$T5c(V zy{u+hCz=qVBUO3f)^ap?7m6c@*9?}hkLpqud@)@GZY@VbT?%^(R(z(6hl`XO3euy0 z)SsP*%GGL#xlUe1N(9yN6u*Unq8jvE7o%jWrT!=ohyGqeDXiNN%1Q1XzYgOeiPCkz zEQx{kmY?2YS?rAR&XhIM{P(u0?856eAQm#NY$e-!4qnWrjl7%bCVMYLZ%g#&Nbj z-4iruI>8eorsKAeZ20QJz!2r4b)rqZHd%@9&*yi8mG?uqGh#qPoe0-sFlx>5bKVuUNpgcLO#V@ZlUN7<4Ei1hS>VjA__p& z1%851be1<-=_r(B@P;$OJgtQqNMr{7N4(Ra(O0c`%I1^3v(9^iBF}R%ld@V_VuN<; znZ(gpV=bgF$h!);+JiTa);QZ~YHqUbr$@UvJAH5EAyGsl>k>(x+KOtEq+1UWEu^-Y zC!h`nOJ|h_2ns7LUY=BFu?bi4)l0)pCgU3PW%TH6u2%^eyJKbAExG#acJ9zw>YBr_tk->O;2l~e^}MXL4c572Mav{NRWV1r3xQmlprPT zLg3JN%WGp(AWTj@o>eMjck}j=hghtxmf8rkiRBP1e{G;ut&NjFF;V{WX`$vI50bP$ z6QTn=Cm5(xJGHAFHjAT5mXu%yc9y-!>8UKAvDk2E2nFB#{dulDC}%dN#Zj>3Rzn{4 zq5aSiQ&Ag$mfGHRPYWjG3{Z`JZI1=$8hix}!V>Yg8A2J3;{urk22g;vbmyrFP@+?d zgxfmN9O6JSn(FASu{QL>)=Y_Us_9O-07n(Gnqt`6@Bn?0x#QHNJLnNZ$M|iqVdwe7 zpVXp+{pHG*S%`DVWf%FZ7AaxRzx^H#y%VitB_(v;$<$83)$3ZW6CXuca0P}0BB5Ts zyZt#fzCT^z34Z|gNArGlZKYhOhZiD&^QQ)|ZF$XS&xTCwKzrg__`WfX^}hrGvzy85 zw|~-UFM*!NxZ5ah#_*`OKU#s(XmO`PlB5qyik#=3>7;?UgN^&laW_*XK%y)tZG%4L z7BNcz&BK07Tbh?_A69N4B-X{PY?0%m5$Ex$0?2O1zHLR3QC1#L#w@7`!V2c{9n38h zOwa1R8G2`eB#L5#j!Fk~(A|lO(!+KBo=jssgz@sb^FfCrFIe%_%%66r5QC#}aCgx4r2CLD{S5S`K#<8O$I3ONhAXIZCKxj#sJJV(9=MZtik1eMo` zF`vSNT*d~~;WF1@h_I<*SYKCzPphsNA5o0is&O_CEkqdb3U;#GAhpdMqLWAiqFajx zfx>DQc)+4VhT^eM@f4J=glp5S7pZpnrXP%MqNku}3dKpTpA57_0~$D5R$K9sXa<7n zi|@xAs#AVhvTDo}A!Z%CcU?CSxW;hEKdm8NKd#000Seuus4S>&s>2<5pHfTM;f@;o zu)T^-bUPfo_i6LC>1$MCv}J8??l~J0=86z^0`-o1z%5k*bfqMy*E|#lu zm|3M>>kw!|Sxz*^ZIg;FE+S&)F)OMKE9NdlQLbN90Jp@ze81CjDIe0bW=5ZyW(Ygb zD>o}AExJ4VMi;K!%Ps%|N>XHYRY&z`M8dYpB*uB44mRN&c-W9$>U*%`8;aBI-xY z0HoWet`B&5U#P7kicb_56>qrpDh+3yrh?3sU6!cmZ#aSZP?wYt`Xc0gm)i|}{`+;1 zww{W2JT0uP`FA#KWf^aZ5Fo&Y-BvX*< zeF0Swx#!0d`VQF+t3FDVLXx6|lDzO;%54|UHgL|^L?miRU94b1RZl`|Fu9*fAtKl( zAzBF}3cF=(jeKA*bq_jmp_e4Sph3~3*jaY^Y01;9-^58p4z+hCj@D8*rf??(>L`Uo zYAA{BVGWZ7`@<=h=$77+&FK3_aaSnkO8XVU|5FIqShVmoLRvIhc{kURfe!JUhBaY} zMJ?${+!r`2y25`mGPcFcYbMUevLUl-Vskhh`tNkbZYCT?slT=&cvfa2c`&5o=WXoO zXcf4r5O>CGc;aLl6<^YnjKBCQBC}0i9xqO0*Q_YFAW)5wxrPEoePAR-@KO|Ie6(;$ zZESt=^kzs9#6g=&Ov@%=7cgSz0)5!in^AHE4uJ$a4 zWZ1nt?}>{eZ4%oKYRqXqGXu z`O@)qLWSwJwBOH&x2aJh8`JbfKYhS`aieLAj$>_Hd2LTp(%@{3NBBgTABQe7@h~w7 zw)KPpA>xPe%J7r;^=FyoJ$X`(27ai2r?*}bcJl`l)uc(FD>(4y0y|6(P?GV8ZUCjK zVsi~FC1luWky{Hkeh53rZuXo=^MjTnhJgLTZ&=Pv=DGfm{Gpct0FZd`(-pU*3P`K~ zqad%oswY`laQDI*CVT<0skkqoZtY}edTrp;;UioOj{l-1O}_mG4@Lk=%@S%NUXSF) zCzWRH1*x9Mo39tA0(8z(0waN|+@n3P#=KPoVF3k|nW9`}CXn2m6*|t@s$Boyxx`@a z8YId?gSm(e>X#9Mb$JE|YHRwV3gOb4#zFt;E5&uj<8VUg030INjv8N8rNI+4k4mB+ zQ5Le#dbg3~%40P5Kv!ekO6fsh$+Fu#WJJLteU>k~GY14Z=Ly8m4A*-=18yj>FaqRk zGKqSw{&^G;4lH+|0Hprh6gv^&-K&vy%q?}-MJJ<5Gop!ZI;v6q6n|NNy26jRLl4<1 zR8MMPX88bWoL56kO4b+* zfE{z^z?hTUrgY8(Y64umoVt{_rP1V45eV!{$DlF}Mp!kht{gkYYX~> z4)MB;GEGKOZx`BFWeiP7vM5q@n6~|w(XuX>xPoi9-8`v7*~x-n(Nuc6!|7A!R!m%h&%eIK2pJGy=gKXUd|68%5z?k6MbuNScjow&Sv1Q`@l*5R5O9h}I1$G_h$6zOLWowL? zy?kPJLUm*IFj6=L<-H~A4G3lvb?YkVMD%T|77eir-%>kNTRWj+zqUndnb)W7G+Jcs zOl~>Jed?PR&IzO*9CFwEzlY(?sc3bA_-7H3m2^3&?lQA)bj+^(R=`l5%fY0TNJ+o( zAOcEkXCU)pbuX#1ETM25o~%~O%?kqPE;;u+!;?b54(4C!#J#e4{NG&G}QZZuBX^p7jcylG0D(l&{y_ zbG1oV9xlMu>&RH#6c+mMpEtdyKr6KU5$oRbHWz2G@?^G=Q%-*b8k3_C<~zPzv!6$f ze~#w|UZP#yt37~UZ{thIkd_BzKC_&Nm9GhMt6%+Ihme<|`>p$K$AenMagTlv*YyoFmZM8a&m13=8Oyv%t>Ft(;MGIonSM0a*MuM!;NPt^f;yYSl_8UDVrAd z?G3bHi@fh&_6%Gc5h_&df1-ZvgBknKRyte-~=t zj4rSwt#e6%8?(DUZ+`a~LVX_d9*<9(&N*YxA zR9`l|g1c}0L$vkMmjU>D^Yy=ye!d>j=_h%21)fyPuG0)yuk}B^d^K-P{I$7rTQN|A z|MG`%X=7dilskKQWaUI_#xa331yM^0w^8sNE$yGQ+SAFIFwaC*c&6Ig_%;1jiSchP z*Z9fF)6vcE-VgN_%_Xuw_gZPbq-kN>cRO-Z|+Hrl_hZxaX$DG-lcMn;rf_g==;8?DjDwK zdn8E#mjFf~l8jBWRMYr8Y3o*ptyZ8_Ri!PfXM8{A?1=(s+N(lv1eyfrIH;08gtfcO zc$c{I{M~ceQ9@#9QHO`xSh#v)?(*xNj_!n7k8P;jo+K4Foh(=W;@3?-cV3%`ID2Bw zvnhaHo??;4!k+q)jO0-zj|T0WUSj^Kv(}jZP%EdjULi`gPF1R=C)JK$bs1^r^{!~j z8DiWN&m03*G%R9_OclaW5s1fk$fD9R^zO6{nxqj%v{UOZlwlEK&1c9#`{sIs^xix~ zOkTHQtfA|k%DorkC(!&-4}Dd&J^WPX6^AA_FOr5Q*XPdSkRp}yqHx2OGH)PfB0RlSOZwa_~O!yqbW>9U@8JcNW8glC@-e* z->isFWcl>X;VTbFkx#4@jJgKHSS=cl4i_WK7qi+QTA^^T?_#lAJyEW1OSshw!GSCe zA#N#^viKaa3T`Lg&n+;%BW53#0D!3SM+JDEWp|G zB1|1OXKro7&LMBD#!Gm{>!;EpZjp8ks0Hx&PDfS&&(pv2+aPhXMaxxG+7psWTQ)L{ zPqYz#-Z)SfQ!gO+_37kY zU{pso&NIs;oHgrj#b7o6jx4$SiYO;^`v4#A$)f;qQ7L0>-}Ze@Id2#Z5vD(;^e{V-JYW> zN}dXjG+T$c{f>s$r2!Z4an7KC?#^?1EjI<}X9&OODd4~)kFOfNJ5BT7X(9!* zCFOGg`a9)Htx}D^c_KB%$rlC|d)dK2C>EO<2U3E0m`O}G=yoHqRZ^*VfdqMs0rb|n zxDKhcn#O>{-^N@?p1O#Up>B2ISJ*DzcGV7ff{?>%_AShYp`erCv!>Ma-i(of@sF)3 zuNq&1Pw4R|wTkTV^(gP~HL7m#mYu0Wvd%VfNMB!mj%n$#)Y27J{us!3K#MB_mUEtu z8c=kVfX-E=X5oxUYM(1V^kmJlBrVU%XdXC@S82?&D7G?LgPa9wv2r~Pu4(4$c#(-_ zy=+3Hx2!K{T@%L{ex4|ZG%$-6kT_N8C~TLoxC~l0Gv<;@yX{*K(l_i0vxjtL-=xba zA+{3B*wr^C$>+jl?dX(u^r45gF!ki66R8SguF|p@gQC$x-J{nvTOPT44pf)abb;FI{86wfe;`S@?yJ&P*#nxI( zjEEV6U*jBN&8RiP%YbL!;un@igr1yT4`6$-GaY>UoJ;#ttsSJ0W96v0{d9UOYE$p2 z42(O}Ib&eb_oZV}r)|6KB5)<2fi8=JkP`?dK9{%oZw|%=)Xh?;-y=zIho`4mV&`aH zIe99*jI``GBeP^aR>=Eh_6_g@?mg>xlao)MzRpfy#am2BnYO7imwgdqMJBz>y#|50 zav_9{is$y9Bw_?qjq7fpFyr5dNwd#qIw~a&zYfvZu=d;G@{SJ*taunH*XxBLi5FO= zteVO-CAduztmA?xgxkg557ue5;2ygq8JoJZl}Ep=%(El3OEf1%w-sJg$`)!GaR|F` z0biEbi*6-@5%Ucd;vDlgN6vsoWROWK%XDu?tRWM8-pva_%*%6P#4qF6VDeLiHUUM= zNTDYh44m2eBM59;J&g#_kF|OF151#nnKL9Zx~;+AP!0=u4=PN!fnwTwms9=qtutcb zlFs?<5>So>pr*X7RdCg?0A-Azp(wfL`oej8hdbbBlFr&A%#5bjZW|jLTf)^xns!ccij$*jURK|agQIIM9**@(hyg8( z>1*WtHvnkl0_Xg7fp)(DB)NMa+UB_~x~^xz8gM0&K7hU~(O5h@NW9Vz2c1ibyNy~a zp$xK2B)g|8!E->JK%CQEKETZWNpaw-hT8GUhs(_%P*FcvoD2M|rc= z_~0A%&G=ZJo0SEt+=_BB9WQ0aV+1JziI_5;43nc~hvL+4e5b~+LIoq?Bj>z{P+n#f zSN%2aS-kw9bU|ZD626W{LMz~jUaVb}JpW*ID!6_-{btC4)K#y;fH^k}B5P^Ns1A7m zXWo!UQ*U03o4#5;IfD=6cc%V+T8VkU92H#VRSXP0|g6qaXgy)Wc(z=ryoE_%2QDM}zRy-Sy79skb`phzeJ?**))WaHA=NR1Jq@~5VJe* zHOv8uqh@baWbPPdM?@}EX>}_Ob*Mp2v>P^&0X)Nl?+_a~GMdCSwWu4|2;P6ZwU808 z%WJuTjDuWXfeUxNQ`nzcp0G7oyA)lowA&=r%)#kI3EbT!;VMvSzZpE)K<_F>xI@dL zDHg~IKSDk&Jo(uO#!6_|6~+e1m`vR-*CygD@7`~*txY6la|4x@ojyJ9KC5V#;YGY* z-a}All-s*}$B(bp_xo}D9t>$}!vT8Bsv$*TF_cAH_yg7#Yt7>A^2u)XV_kb-)FsrL zy9U&a%WYZFb9q2M9yF@M(x2$rcBF`P%k$WC0D~^YpbMOG2w~pK0JNsa{j=}$^Svu2 zkFXWfj%|M~aP1aEKk_S*N;Xw@N1R8sY|h$>nf zW6Ds<6iJ;4hT-KT4rwS)qb-XzjXwVMvT44};%DR^fS}8cAFd7(e?)KQ5_$AqM2H#g zWdQS1Haos1-|-X-)G2u_ReW&F09?t$lBc;`6mTbHalu#nYJkDLB$d%E%`*11W7 zIIhF(y&>A()ECkd?nH183&LrPhofcr6iH60l zl*iL(@Q{Hq+NSru&iv4vKhR#!_>gidUaJ?^7U;`hnJ+XFVekIL#iq6 zRMWUAq%G>GcFNhG%83{mo-W3x!Ul(4N|N;U|C|!qD3cvw;g(!1xvr5>FdYX^-l@j) zCRwg-qDpS!2JdfQs)JQDb?22d1@%U6@PR*~D^^h4Q*EyzhHk8|>cgQ#18r4)%5JMd zG8o4@k%*lddL_SMaBX&&%CbN+B?1=TvXOW2SA$N_-~|W%*Xd!V8T+2M5&?)(9=ZNY z2ma%j*c%)B<~9GbjpUDeWCsfOBt=% z3Px0WW(dF;~2AW{0h?~iBQG20z}LC@CAyma$rbcG9$Q{%4I9`jtCw3`kFxl z*WmgL6#3OM1DZ6_hurQP6+E*7fpt97pQr>Gu2CURy7%KGTTu@>9kZ!fDZ=d|ZY@KN zJyc8Dx{t>8_TJ`~vNV;$eamJo$JRudnI~1w&PAmdYVvvP%$L95GSkLuq zeTj%F)9sty-vcJ~Wg!z)NEw5|Sk6>c+qD72wxqJMJ(o9z$|ZUQj~8EqB8N9n7QR3jJR;kQKQ3i-tR7$7h9-jK)3>jx*?9DY4}! zKCXhhE?KY7Z_3+ein|3>y+gxm{4a?sjZlf3$F^=X$P-9 zsg#y`PEr=qMC#9{MkE(N$HShxu~9kK8n0G3a6?Em?|%a7%jPJb)w`1>cvgtgW{$%Q zE>DQ6mLS>>3C1UF1EUdVGQ@4%5FKbY0yFkAzNA!!cx?>96>LiSN)YyPaaIeyFxtum zlvpoSc2XS%O2>9GbFkAWZa5hje7P6$&6Lokq0AG9IVmnE>LQs2>sdU*pM);G8`(3| z(ir<_d!z~WWMilBS8H++3TpFxgaFR=*F!?ap$e-G2^NY?S_$G1V*yy`kMf+Mrc&h4 zsD{koj^cRuQ*blFhXR9{j#!+)rPd~;)8=|CEtB}&g8Qc>8o#Ymc z-7M#f@0xOy+G3@tYF1t1^kzkaK6Q&u;+09nD4>U3%M0#)m;MoFB%x;GYdzzntc{S4 z(8)fL=St-*?LXZ$?PiGuwsy@XR~E~^ej{n)O)t+L#l>8+uuvYpTNR3%_~eDbAoBM- zuCRD3`IETW9tCpG`Qw3@pS1uP^d^**E9en9SnBTP#wX2SepLM(e4iBEyg0acT?~8* zc!3%DCKo_K?Wv_pL_3SrWgvgXT z+t*WXuGOW!o>Mv2N7|gC(NO-UyoE9O`8d%}(+2hU=`;SJO=)en>h0(7@crWGNWDwk zupfn5b2e90a1genN?Qj>@{Z(zbdKaXeyc#~0p@1?j8O!pL5YveU&Fo1uDM&l^^IeB zb$?UXqEk8uK{ct#XW=go?6tD>QhmY+K+gM4w%|%el+PTS zekJ!9XPf~&=Xm0qJDzAVWo(DsN@XRr_M@nsbqs7xczIjiCQ?#~P_}ZNm4uTz(eA(n zHU@*4>MOXVN8oVuOuKIc&@<^Nzwf*CudAC2Qx{HEJ9;_zzA1Y8dGWXL@75|Qy_mh zuw7ML$GaVPokwrQv9>cw#5dbLzr%td$cN7QLpIMWq{O{@f!&g2e5c_(P#GL0olOjF-#HioT{ zj!fi@h3Q)@s*Vpn4J&nGG!HD|*;cGCpR6Wi%b^gcks-v;rR^o+mic{VUnD!pE9d$c&*U*+p1w%IRK>zxgeV{ zfm&b+$jENCvLxdZ+Ik8J-0JHc{v{kjpodM4P>W;M&31uL<#1daVLecz^As)sZm$dbc8WmW{vkFew{@;<>qn)}p7+R;bM<^lT=nlbyw?ft7JirgXEX;pZ<7 zX@KK#+#YCt8W_33OW?xmQb7e#m*CnBEBN@pd|<>#I0AZcTzD&ghqE+!*j%aQtPzd2 z5Z`D6*f5B&)dM?M0L26D%DA=i4>G&sBJVJP!)w7*}tF8b4_Qx4Kt8eJ3i zpjOEry&q5~5$F$FXtNAO1A%@Uxkn05+roE)3^=kPY}YOadoDO*-u;$OSDab4k)ek1 zG{87BIyQdp=V}A-l;|8&Z~1p6&{t2(RRNK!Xl86SSx+yzhg&vs&1F|&)~&P6=Lv#F zLnM$L^OEM%d)==)9xRTvuGxnG?P0beINOiw{3_L_$8#$t_#sW5?xc6u<0(ZQ*I%NO z2*8L<)lG>xDdTaq(8;{59Wk!su}wGJArD$l-FEdNaGa0fP>U0p66rFqM5D2oJ6%nR zm-8=EG$P$I(PnEG8>6{<6Y)&lCSNd^2!o3bMamzh6Aw zzDv5n>>F}y&xoOCSG_U2C3|b{rJX1nFlbPJr3kD0FyxcokMI}ilEZ*tW|Rr(Nm^m2 z<_@glyr3jo_o>gn+w}nBfQ}`Ig>$%ZfFhpOUtOuCJjgzWK;q`+=jYk;=3Z~!%-0O0 zka-k@n(3*P(z_B3vu>qQW%i=dN;XIzKxhd=2z(2;UcMw^U#Ou)lO zV~S%t7u|^vh0g$2R0)I(!e<#NBHyJp!|#aVN()Q<)8zav4(~@|e4h)ijR)(Jqve@l z2=%7eA3PBXxW)J?Ex{hvq#yjl47LF|HKTM-n1zYXA4p%Z8()}7FJGq}{rm=L+9S<` zmoTMv*hwE5Z`7ppGb_q~J9*6g8KKNeOSBqiWduful3SYS*9n@p;L8~cqZQ5J$I>aGV_iRi z(JPnmcbM``1AGzdJ5b0=LK@_znXlFF*GPK*kdB7 z!b4J6Y^MN|#;I8GxnystK}>A6v8S-0u2P=s$&LNy2l8Y#7LfdZZircEnwHaf+g!d; z)VMN4&0A)#pkSv0QzR(KS@Ll87Gs* zSW@TmRzrUmZb)@8Ln>p%d*v1vFkk1_8 zw(*Y&Y$J`5;3?6Bn6a`+C)dSk2H|RTA;pCbB+du8j@P3Pz;*@lP?v@3GF~jyPti|Y zZLMYDS3|>Imy$BKgA2jsltx62%j);vlGY2ecU(gBEEt?3@f(RRL9xSPC9nWhz{Lf- zRpZf6?zfY9CBKk}0U`i8 zJ8VGB=CToT*tZE(Hs>D=0B+I!Mr9ubsW}eNNTi&B`$KqDqt)8rD^sO&rfe*koS`xE zG`h+n|CW@?!L{Eq3-!IO33XqyPb4G4Gq3;bUR1#XW`&t?CZ;P8z98@=1X z$==2OA4u;1flknhW&IBz_t@-3DuvmY6df zL@^QU?NRa&M&CG79!{{ANMwxN#HZDKd`c}k(imWtfcPdZImdS{zlQRFlI$bz9MW{l z_6Hd4+$6iiWZWoi>Ca@L1ia)}O~mP^iLeh=*g%!|nyB2JrWXl8=GGg+i8s=c%eaJ& zh;^D-NwXOr=paqPvxLPI3uYI=wkm>hK~c#igZcUM$Wzyb+a&O8ME;0ogCS)E?uZa9 z%XBA#8p1hEtfj;n&Ff!8S^h$*Cf~;awnrOIa-=31eiH=k2F59gCQ{8m?8sLzL}v*~ z`q@AwvK;-iFV2FoNeq*;J1Bv45^3cF^vC4J`QN;;?gQ<_tEkHnFA?}k2DCqY<(2~L zgw1~#uMyKe;jRobn6ifFNXV28Tpc%w<>%?=XshL+2~uAtQDp^_5=6zFWCwXzCTOsE z%(BT4A9`#P(J7yJkVL{sW+f59{hAE*FitEa3`Ey8k0UlC={{UVOwIxKV^zr=NCZT84?ewUwLajfB!(sH z;3-7L?&yp@G&u7bP7mg+`*}g9`4QRS5T6$pq;+q~>3=d)Mi|f5>vam3ws(o`=(L3` zPllw4DMPkr%b(kMf6R?M-bRZ!>BH=KLc9BFUiDo$GP9F!JuBmpHc0Z)upXI=58#LTOCy?z# z*;w=7@+{07Z*yhMjoTd*p{)(ve7>`~sY-=y!jH29H*R%_HP47=k;=3kwA|_B##G~0 z-o~0aclhH37^2%D{n7#MC>DLi*;2bbD8s9?owXB7UDs$k3pUITPEQ2~&M@cEPyq*< z*5R5?U%qZd3Ph!Rzt-F9)Om}%?Zu0$`o~*gc_|k@ZCOg5``{U_!oj=0_GI75bl#)_ zB>xc1sO@y_Tv_g$Ip>cZhYLEi4_DFsw+I>npHu%tGt$|lGk{#Auh?@lrHCIFa4lCo z&&RAk8J@>Vjy`e9B(|KrJ@_AQGBm{*W`_@rWsykZ1T_T|uM82yklYz8xB)QI$)$In z5;tb2@pae*G(iW`0pYu_id8P50Ep;x_v!1)>_~OJI4-p?jUC>r1O&nZTVqf1>pv)m z4hfe%lB_}-((XLg0vtWt$70EpdqC?r$a7Pc`1$$$Bezoe1WKaA!v@x9Rq9QUgGzER`xzTp?I)vfuI~+yXCP zt&N$URal1v%`hH+9_{!7rJFP&P>7P^WB!;PSs17q<{d{i&}Fcw1fI@(K_xFbaE&o| z138YzJK&atwJ~lvf?KyPw!uUY$+sVgkjB-d$Z=8@r8stip6B7ac(v#3K0rgW`%gv( z(Dz8Gj*8~H2w~N-r}*3@8t2O-Ww?rxR;P=jZt=oGk14+E%R%(YZCxgvYaEw=HlNz5 z+O*n^GW87OcO&W2H*dL3eC+L%tb5pI7j&?)tEd`bcU{o)s2;78)JP4t&2BRkvf63o zZnPCgcs|YaWbg`?>gP{54eC&rLVM?UNQjBwb{6JZd*v7-6SmGYq<;% zMhF*h&JeG59^Nc=$BeQz5wSWo9aNeC2etpG^t@Q2FVg)DJp(< zjUON{B9-{j6NuW2*47*pBqHlfvHK|?g@_h$O8Iu5B4~|iJe+-&*P!5PK#mH2YoNV6 zd#Z6{qQyPAgb?D`KhND|7HD;AD`V~~up z5UKu=?3LCeX>C#fiz|sV#5tRFd+7^+;%ETGYnO$ifraBvY&1LVkA%8>al<_6kIM!2;pf0LX_z85sm}%{Ps{p`lS2 z4yFaSN0F)^Epn50rz%C!AKnUV`JKg>;xlF*YoeJ8c@lSWOSI4TF?%=FhIuVPf-s#N z#TZQ8L^8OBmFzb{TvUS7jd&Syg=xz0 z&$0Hx&FP8Mp$6-ik;R&4J(+&4Yx;gC>xZp#z`Ei|H@;M0&ssG$GmwyuG7(@{z63CQdb}gU2{E)0c48XM&nNMXbrf`QQIK2D4oC4;0{f%ub}0 zDD#~_Wpy+v)f)PY<4slODbxf>gjj;;R6?%lB=in)j~`o*tffU5+hl*x!FZs-CDcHMAjz@wlUksu+2b{E)gr$>T`F@oRfr~ z5`gi<=P-}~`x2>!SZ>OF3COCbls z2m1*JFvPQcwg%M{EvtyEmDY6>b1Y+c;(mxIy@pGdS~CgZIG?J5Qg~cJX+j{Xt6{j< zW^k6hxU-u;JYN10_X32`2VGjHW6zTDeZ@Q(N=8JJfX4YC?}(=_QG76y6-~*W8CEeM zD`~>pNs7%hPf-_RF*QUMRoF)3OJWm{b2KW7+Z13@(!HNk% zTa)1D@jb2*k%3B)$119+9IL+$W^!`z6qy>-X#Ud_O){2Lp$wVHy1_ zD&m3KHX27CEZrj38q@mEL~N$K`xDc*fFKlo`j6`8`F|DpK1Au0O$4#=-|Tjn^b!d<^ln{ba)EhKsO1^k}wmAY{8 z$i4CSGV=EI{8c)P-GC%Jk<^4cB9i7MnJ7GXkh)29I|0h0B2SvAk}ndK%fkzG0#*Cv z0Rc=0)})(JQ+QchoZ37oh#iU&7iWg9?=RT_6UFm%5Uw}@@s=JLqbxlVuEYdQoMv3M zZ>$7BjN?H`mcYwr^rCzsc=3mQ_$^PlWlT$I{E5S45JR)}&77|TG6ii4yS_Sk1e=LNXRT&H!@g00cj%YYbfUouTE=&W3$gGk~=H8qW)&zZ&1K z?p_UN4UIHsEgHFf;jb)sx-kCm6fil+Nv{)6^6M;8OFa z37FTQ_x%3qO%TTz2=#7lA*%~g^{v>N9&lA;(n(QC{UK)oPRg5(JPxgAWnU~_f{pmL zv8IunRw}C1h{mpVD~S~X4fOyrCEUZ>!c>%#@SIj+f^qsXBMe4f_`hM~+?Y^phgp~O z$+QIj-s-<0Z^@|O5M)w#7DEQw>xwE4{;Fhz3x_TYDi8I6FT}DMKM5uIBaFOdK{C*KR_sb`Qb|5%E zat^SVBjaarwj8vGyfBrcsH!k(N!L;Hfd^3DKy&e}N}Eyr@+VPI4X!(kFYg=?=l(+# zl)$t%a6(@qos@>;&_n?bJPrX63G@#1^wr$Dw7I!X)fC$=5tMy*Di>rIoCl=2Y%0je za=;P+(7)LT&e#SS(5y_SP3Pj+Upb}U+Fi|OCuLsQx@Rd-;m4Ikx_*yA_w3gA;#Xck z*OIK3Sd7IueUMgo^AB>W^hKP0dc{X32)a|~Fu7$=SM+T?O~&7Mg-H?Es+f9n8hOMlqFt3;+MoiWOs1$oxlTrUC*0fcIb5F!FRU)pvGrva~bT|IerY@5dDX z&03)<({b4hD7#On?UP_tkX8#cRT7&mved4NYF1_0il&?nkRoGcucXL)mHUhM*Q_8y zOXCD+uP>bAeB<$)G4(B+$|P@y!}gI1MY?%;A}6g!FvXLv3tAc9mR&ktm!-%@G98%)b2mjHiF09ubr0-+`^Q75Ec zBi7*Elrje4oQES;EMh4ika%lL(2MindEKBI;nEjhlEN<5msdO~i?M|)_Z?CEF|y89 z;P*0O;mcOmmE!0)_n_tum(Op_M)AC_#j4RrZZZbsc!8&I{)AXAXWAV~ONTBdj1+_A z`3xcJKOAa|?sjsJ-x={uW%uz)?$Bm(Buiy(mZo_$PEta157g&3L(6MXYtN4=_niBd ze`Tw{hS_-rOa25&+94e~ho!OcV^s=;Zbk0|+CMjo%nv40^1S2kY{abM>-(CE(cf*) zA0&@|BJQ`oC!K$xSZKb78_WViDNZUuGhVm5_K@~XS^k3k*Hf+xp;rLpe=-$yM1X&x z^uOGB|98~?f0Zi#lTcb!-?!doK=AvkPq>4g4T_vSRDhr{xfU+SzR^{#7-EyJUEY)` z^(gdr{T}5v<+@d}o30igNz%RR^NxFW(oJ_=Q_28KiAc(6j>MCnc#$WEdk$Rf(wb0S zVy2`I3~n^{xX6!8RFGnpN>{QVZ6pz*h1Sy0s({QwR=f0y4!-m*kRFjKY0SyleA>k#E1?WGJ$=4pS27wL zS5H+a65p3?Mm>PYJL*-(J|0gG2F;=fvQC*YNi9P_0xNFb7|pFz1rXS^g0~$uyj#1#1s&WIoxGq(MSuC+rgfLYV^3BvRK0Gy~!%&?E&Y$?ha!A*xUj6B0`t zEU_i86+r|SZn~rzZ8^sL7~PmjIC6*UyXa@VYi68oV|zpb7L0(jugF>a@&6) z`q>@oeRB>!5qZ!>JIJHzc65C*kB4qSHpY0-?lc*^$659f?^OEaIsLCG>+NIC12>1q0>>z5|5yhrwVU->hnb;b{-4Lm9o(&`*gyaPSYQAExc@cLto~ysF?4b=^!%S> zTC9q6d^W>>%#a#60zwpolXyg;R^8oIT9v&myrPM$vSg5nj4N!}XZ!9m)B=#S>|^`Z zVS{<+jpnfi4+G4j{Q+kWOeYUEe8xApD6ct}tiJ$Td5T6}$Mgni?pmL5*0J$tat-rnFdGG^S76k4U-)up^O6 zx`%5O{$*R^RV#vAWtWGc)T|`=j$SdPVjw?ZEa?;k^P+XlyjaK#W3Nathxe^v+wMjm zvUwA|yupJ8x=}cPCSbSlJM9(lu8k_?>J{hM<8A-(D7xo}i4mRs(Q1SGB$gk_wrLWz z{i6%?o2s$m1%?+aiz3eAp~cU1HiD1&;Z`1)+H$5AYU~zRcUHwrUt~`-i)>tkK2huI zQ5%fGduE~>&tx8LiCH!;Q@LCvqQ>~%OfG;p8XN?#)v6;qMGEyX5YOU zy=#!wpDaCRxiMe{T)1)6UuIU0Kg1AIixn6Lx z`n+q{8ur5qKIt*IeuB4WQJu8<4(WNBme`h7B(8by$})oLf5Pm|x!c!hIG0pw30%=% z!d~SgP>@JN)P2KVyPQ=&nt%Qe6OI!#7}hm70DuV@006;%6LH3dHa7pUk^WDD?pEuc z5r+-oKSrD`eG?hvuckTyc3@|mcG-0nu^U_5t$+g3EkmM6EK$~%u%ExUmFUSyM=Y9M zcGn>VZ4<{2htA{2xVv`k+nG>OB(MnaizJ8_iP2%-g8j?yf&YM`(c~8a5T0_vShkCV zn+q8YEz6;S)`P?$4_wKa1sKF=q&gBQqci7#qlC#S2@I}?nx#B5B%_@oA`KcwAA9#V1Co-5P-?Plc%Jn8Jp=V+ z{PGUoP2P7zDK(XI^30hI)ERriQpo6(j;Tob;*GDQ$By;Nm){{2KmAH%3V0^U8=phtZ;SFUHe~=-zo<8A}>?XrEMIRuyC9Si|1w4|KsHNh0Y4P#E%x z)a&<)g_B>bP9pL0a{97q7o9?WJY1YgxA^c0KP@L;+B6D?%fDPZwP^%B+Z3Ln^UIbu zZx*Vh6gYPJ;%2xApLWE?>(Sj@@{#*5&uRMfj|oTX2@>v%P;^?l4$rst9PT&wEuL%b zU+k@&hm*)IDsvEDkU(Uh9aTdk(5IC_KzQ^Ws+da+w?=0~)9JWy`i&ZEK~#m;B945c zRG#Tt&n0Z(VpAc2e_bc&^EP}yOB9a}9~S={FB&MkCjFG%rhpt8W_=zLJKT(oN1v@&0LyA~B@{s3^< zrm9XIv{WhXxr)2dBEd$EI|TY4c{H4FZ7^jHagip(zpu)2RC6TuWiYiD{nXSm(c+1u0HDlae`g5 zmbftp^yLU0lR7qddk+UOvw#(zu~?%xYhGCnDa=E0^t5m0N-3e9QJtAmsuuS*=+Ppw_HeNGyv_`rw#4T2MT7Zzc!jE4 z3LA|&t@2;xYMRZdS|;jP(n?&7T?KRuFF?;7X~(VT0LQ9GoL0#dM(V#(1)+QSTiIm^ z!bs&L+{4!Ho=ctBenF!fE`&-XO99Sl`E+i~I+uXP8#JATe(S~WeshGHu>mr*X^95W+1LYS%s z^JspxiXM}^Jc34ecE%#6P*$f26Z(#tbRH=rL8qvP=7q1V_{Ys4 z>Jyv3#QA07)Y1YpHWb`Eq|u-ib{H%G7emv|0T8+r#Cwa2#mm3 z%9*y($@~*!KZHo%{=swgV&$kEp4U~6(fNdKj~rcRzRaZ#pO|+X6-s#I*g^XmzJ*8` zPvQ0O)cBXbk)5lXbNNr(T+a4khK^E-SrCz@9EnzQM#znOsy>L0c%ACRw{-NtVJ|bs zPJ6?mE}EI7eoypsX*RkL(Ftu^Wl3WD18#IgdP)y45{5_ql#-I3PU`B8!%CEjITd~+ zl5c!iE3DH*v#P{xa&@06mS#KWs>8lvWnRCBf!b3oBy??UM?F|Ia>-9+?N^3S7PgEh zOr`o+b=7lNMHge0wML7o&3y?6?#e%&Zcs0Kz}TN~u3N!GFmo!^Qw0E5mK&tTS}03o zW#Pzv?d(EJ4{wj#<`qu0D$P9S3z-$E6FJHfi_+e5!r+8=dbxjp_Pyhmz1+3z$xj6< z;X()whCCoMY;`st=&`iA+1f6S(X~<^(uzs|XMMxjtBchH;lc>bz5&ZUR#A&E0*$}X zUR@@jZp;*9nf@c7%?_}_fXCp6rG#WINxpGC?05Fcvb<~3$QDg+Wb)Ci?39C{#!pH^ zN;y0J>dmR%eS_x@eI)~N|1B~p!ZifNQ)Gz-NXm&9YAY!`KfXrv_3r6?UV<6Jo?E?* zj_M^ev}^u$gb$g{X?$z-C0MHMdtKp=?5|DyO?%tTdT=vZI`!-ANZ)nmH>}h<^xjB^ zD5r%UfgniEN+cbf2iaV1z5UcAe9UCE7^*9uC=Av&xsie~K21r$@ROQn$LX3>yt6@3 z-?luO+0kypC7~?H%j)mf+sxYR`1%Z&UCEbzN~&Jcemk_U_YeJ^YphmM5-bcXC*;`2 z!0q=F%+6JeE!ZbV{M+@%J?4UM2Zb`f?3jUR%&NVuZVd5|QA~Gd)0D9)HOw3Gf2|K6 z=L&So|J8@7|LQ~R|7H_mV(Iih+COMW+aLeanP00TOpOZtGYTZ}p-m7%G+kJ419f{- z1O*tBB#zS1R*{&F|Mva6jU^!$Jh1VmfNwMB_a_i`hn|j@G2Z;?yGBK15lGuMYX^?)3!v)k z7*~G5``CDYsj#;88*eAllO}KMNRuXS{K#F8N6-g2$WcjXGyj*5?UxYE-^c95c&4Yv zybtIFJGF{@YlA<`wqgMdKeb@?N(rLr+S1=nrGyH?n5!8BpYm}qlS58xGyWal{jWj= zZ&$3ZhB~sQ{-qQI)7S2$6GYNcoIwJ1okeSOsq|Qok#q$X`G~l*ZBCFJeq$t)^%8S* zUWx*tGHOQ&kn{B(jko!2yUMEa3*y;dwmuy*Od_OOJaS!M%eLxJxV8I+m%f>%36W+n z-S|DySz|DG%`iH;1rbIWWjDVmb!PIeMUI$g`J&)r?jzO2R!rs9A6P8*TwwB0)7f^U zddXtt*xLq4GYZB^7iu=M-`nJydKII;b||L>tR#HE`9(?DB$zD#DTVSdsHD#K zF0s4N{c^Fm9N|8V$F(H6+;-4sMBFx8@KBWq7A>NUkr)cv(pVZcLA2pEok?s5;2l-8 z{aoUv&n2)_f28Eo=$odRO%@Vmn{^jsZ6~Z%ma2dNV}(HNQ-0#O7a01}`39-?AXU3z z__248hgm%}gsEGdr013aYcH=f;HHUV*3%|^LQ=fY?eqo6f3;lZ#$R@lj-WLM6_s6X zN~w>}>4VS@WixoY7UOi%?Ml&Jtszx9-ui z7r(8|*Q$0IyJC;Dm|o7_Lcp9lj3Jh|^HJBbX;)0;HIjc{#8C>3Fg}I?`yXX2Z^0$f zJ&MR6Qz>1z#zDq$MT7;%D{(_2Me^tAq^Wj8ss7SH{v`U*A)b~>GCe@+d$nUGq=LYKRdQ<6EwLPAo{TEI3 zps}w$43oSLGrAU4 zh@B5d@f6A#GcE$rK`L&jvHVLLKhCC?z}b!oqy)mlgNwi2$%X$OeK}277Q!HNy$sas zG!h(n9W&~d(7Bnl$f&*q+Lj<{^wmX2ufA>z;)pMJeMLM{>(5(wVUOmIA~SeC*Y(h2 zf?}cNA!Wt%g1QQOLFZuv-Ap(9srbJ-`^tbgwrp!0g1fuBJHg#8xVyVsa0mo?WkIt{p?H<*WL84J&F{=Giccl~STliaLszty99o15(G5PE z++Azu9V0LHESJ^Uo1dffoiz(jw1b}d3_Zk$2G1FKu;kFRQ5LzB1?dQ70W;UA^S30q$F0yre2<)Q`qI!S8Ng-4a2DwvXKgj^ z`%IXMXwcgTV|0#;I&ZNb*7mhhWK&kA!`Bo>3avWrwT53Asmo)*j?vK7B9fJu${xIF zkX@v3x(~H|bog*TUu<2`d`8qCg=|PhNJwZKw7=J;Sl^qxCqn0o$I9mQ>B#r!aTQ0& zNT)spUQxrknw_odCJY>%ca@&fv;J1e{DfDx%C1dY#VnfXs~_%uMVz}!0#D?+DfZHs zZ%a~|R*a+a9S3F;K37$6v}|S|M?C?wl2R_h4xpS!uxi5pql|D_%?OgmOK3AivJf+W zWQQ-|3gH%-oDikAw_i@K8*k7!y#S{0_B-XqAQ!+a7Ku#rn>Rjk;inO< z12Gr*34LXOXHEl%hf}rMw5lVE#7yF3`B}M<^Y${6Vs$4SJzVCOT+1ZmmEtDb$qmnHU zx@k1EU>LR8B+x+^bTNsd@o_ZR`G)z(f-FuLD8{A}n2b9f@`0$JZ(GW>!_FMhgSW$O z^2)W3i(RFqVb{CbS4+1_x+);G!PoEraGvi|cLVAx=co*4-%vxM+*AniKXG1fz7pkm z)PVRfqW+*(Ys~EYqlsm5ypEC49&Kjf{6qSjce_6Eam77k+wrktj0gM z9|F5u6?VtK!Dwb?Nio*=Y`VNHmt$ZMmt2EkATdze4fAwu>2v6Jvef|nYq1f84)A7? ze?p^Ftj$XKH$#h;6jgsZduC%<7uA1UC#}BPlLV$e*Ade5oAVOx?NIEG$_w3M7k1N4 zLl8vjN}j&0;edl5FOILECElB66W;kTSPBLOuDiJ8h3#3F`q=H9Grauv(we73jH&{{^Nf1z z2$I`vwP^LnFm=cu!>0Kk(Ao)06p&p_XZgl5Ft9h1t zLokf^l)BEt3c#uzWvtH1$x6T1E5nz)jj~h^b<&gqJI4?Q1;QRIptkl3BlggmBkyZc zgdNc?4rfu(Y2lq0+5oBKz};a~m#V?Km;< zo69($)5|7!Io|LpcB)g4Rv<`S#I%#W74zQGxMnP+gK&r~p(b0fj_I%{GKwIt{EHxW zWNGFFWTTvGcazQ2hdwWpZxD0FTB(Omx5{#7SBI>Y6$xi~ep z7RQ`I^0#_kQJeS>V$30Af+BLGq7)^PhNLV~yl4%gEP8{R3e=aogAicGg>ky-qm;q8 z8HJx;LBSoFzO)uw(G2-yJI_)?yCGM~EUk5NSkpa4q6-fd)>f0q;Kq~D-OkKnt*;U& zB^IQnsrv*+2F~^Q9h#~(lP)K<>Y1q1hEpjd@IO}w43jA3p~2ufVh0YOVrOj@&fcbS z_N|vAUMe{aqNR`l_>^3=3h+&R61ahZ%dM{dmRCE_4l%HHyQ#OpPNhKjO-9BHP!Csl{MNXF7yBbaTt^{24S#_+9Pf zacFB=WF~8BEH26xTbpJhvOYBwYDL>T?Yb9j!nrKgWJha@3f@)0fF5zFc~+t&&t=Sim1Y zG+A-YE&iA^DtluButnd&d11TPZ?fEQkrfu~VDi8u%JaCudsTQ$UOQ%Th#w{(`|S>> zYOwcopI!zuYzFx-+a8cmOS#*YHar=y3kPM@3@B3nt@%Rmj7ZmZ>)s9K+m#-dzey$& z+@o9kXO~lwa+GG>$HS}9wlT{cKz(QYwI`n=e0GvM$XB=TByW2vJ{fcvM&cA!2fJ(O zX`{(+6~3s1GCFW!xh*zZlvZ(Swql^=TXj0m>vmZ8p0epPJu~atuo>j)V^z6eRx^1? zJ~owBNV=BaXm(r9UnJ!0Aez_uH6G?0waj4)<2o0+#7Q%D%O#6!Nx;pVix^ntqGKsg z;`%zn8RgjqW}HJ!_KDIcUOby2p$0=ONF@=GAVRShB92dqB{(wFjIe1W#n{Cnipp^3 zC{nJ-GqI<5*5UiNa`j$ULesYiL_HJ<5o;U)gp^qWwOs!AL;->QnL$!Ioc^*7*NKwc z-R-Agc_L|nv0Ndhka*VM_u&2q*i-P=?>b4ipJP`koyUs3dKplFi}3eWV>OE{()ac? ze$Y>L;VtZ|ov2NDj9ahi^~MFF-BYdqL_s(tp;{efiRh+i*I$4RkPkz{qQLDvY|ZoI z_f|=g5=rw0y|kp$q_)xN9Khp=L3g1hW|rbUN!uoMh5;)msNY#l=*oh)HQ&@G@O&XQ zjvGtaD90$WYv@&o2uXAd60Kc2uwB&%A+%g>%WOGD2Rt8Y%Nxd^+vQ@USsFBpHu38*XT$c--T=Ot})?8x1<+N}*M|%*L4hG2mS@@MB zGsQDaZrQp}D^}=Rz$8n5Zd)Iok=mhU2o_$%WZ$LLk3hweFt(0yO5P`+@p0vfQP|Yo zE{{m{GiXCZA>Ty^zBDP(2V`=xiPx;9_(`tWFfys2hJS9VA*9eH|r%HWR1@`bbRm*YmDzS zfXHW}!`%#r*5~#$UqOM~VnE`PfO#~aa@1}*B-{WVxYge^@Igj~sO1U#VQ?ji0A9=^TA!pLA+3WoH9iFI2Gkt8f8Tp>&a^ig>~o1!lxywG;W$QO9_Bh zz22ge`6EObNdnW-$95nMlp1F`S=~wm8gNs@u}!8h{_QU++W-(hcl3Syo8dsw9}WCD zJGX(-Rt)iur(cAzQNtcUT#JJrSQIVhi!r25&~si2C@;P9S@fl08fMtkEz5@pk*)hn zEwNWJ)vQ-wI;x990R5Op&#u!|=yY5-NRF=WN4+|{FRdrth;J@%J_>kGG1f1-S;<4j zH7Hk&&_yn-el9M~$Ja#Oo8mI_TG+sSydipE1=o`t@RvYIfQX$lS?QfB)hDC4TMh*_ zzfmVAh5!*UV+fjrzqUo#ERXnkQX|Z;p4onPU?txawYIgLNM{vcD6))*To@BoQL-W{ z52@jiO#+I|zAWDdTUfbj+NQWVL*|)>e#tkDlu8+0oP?yr9eyKa+;-*=rJSRvPaQuQ zVo<-QUK=l9Z(qX|1q)9D#Kx<$p zn~BKg5chs3F|zbWt*&SdO;V0qWM4ZqLXa@)$9+L{MR2Z4wWxS9e45X*Jho6v3+^mJ zjCB{wK=&F=Ms?6-R|h3*cH#w7%cE&D27T)b=r{3Z6F5}`k9`IX&ala@T1H5I<}WH1 zv#Vc7D1iM4^>`Mxm_6cn(DQ&P#YgJt8xcWKaMGsWIjGE?$bS^|oqrySqJha$w|~n$ z+>r~3&&dW+wN$}wxJV$phl4-iT=w8zj38fwRaaC5VE9V#Ca#%EBYBwn*TARFVZ}8E zr5GbEiw%&B+z!{Ydk0OH%VGn?K&(DRGS!7&#D|tODb=z|KYVHie2Pu*hovQQ18jB2 z_!MT#OMcb_w<+X%zHZ|=H*mVleSgrsp>C!9wzEi_JVK=2$U2S}vJ&aEH+pm_$C_Aq z9leK&d4?s<=9oc;IYpv+(V$44?N$#a;qw5#zD43b72^EIvxSe1U-u-4lZOYMx3=~5 zql9uNj&##d>Ak>+1pUq%kZdp_Y)n_T3Q}%ql~_jPwD?s-00tMxkRoZ?f6& zAhcNtqHoKxt0Jt1_xT|yB|LsO3j0=TFO~3{WS%4rZM4~i(f39uTFjy215oc?-xcuA zw$r?5@_J<1G)i3sp|(vvC77wWMP581m{6}@Y8yXoLzxDHOFD9m;Dl9vF%-4fth1w4 zFihk%SgAf%^Qb39^V`m{1x?G*Vrp@07gK?u7ZdHAMIRz~A*jS8((~RawIuEvcx8zh zV2BZ}YpRV#wv{;NZkO@BUh&DUv5*cNV@9Fwb8H1M-Chqr+(;0)K$|UlFqvKN?(LA4 zZ=wK|_mRbl)6_L41c2dZVg~J@Ww4$*)deHc0|5t&1!SR3T8c>R%05F`c}RwZiQrX32TL0mqEHES#5GJC>R8n12GodVcv>%;Z-P6{X?qZxt3V zJLt05ch5SRzu`SE+W2L0e`AXUtwbOFFqm0yo76MX`^vH#H<^W!dabxyr0C0}nQzo1 zo1k8)Hw(<*7>fs+;a(C z)+_YTL1%7|gl*))x>>@$K%n;ML8xt(t;e+VtlQ-AH2g8KO|m-T&I(-84cHC+bJEkY zp-)m0+UQCEdtZl@g4n_3O^vWW<%=naXhMpD<);IUm&e2tt&xl5P7x*@{iDxuBVtpg`x?`o!7*QkEvgi6@h}VZznEpPN^h!dNZU3;$`ioiA+AZ zy-T@;W(BN^>aYpo*GS^e*xC6!{iE8hpOjm?V7ll70sXQJ0g|$0nTafq+o8pj`lVWU zW*h3K55?D1lRuwv$#Ou#51@X`0+=&ZZC7nia2{6TH-1_g!$?_aD-fKJrGt!Y_z=l| z-Zgp2&-)<;M=4mP7+Y3eA}u(8Cozf1X}zGDOD<4PPSwwbfgpQW)F6(7ordIO!=;?U zI>=`~_X;PbK!8v6{Go^Tb#Q>Q$FmrZHBV&k%qM0Yx|FPN5{wlKZDD)U(X{Xf&}O95 zg1G1a5;EQ^UeY5McE)CD4Q6Xty!6)v{Sdb~CFEs_6y99fz3-Xc@)K%QA%rHA6fyFIpOu;WR%sbp;1ltA^=ZM{w@wK4MlCBJ3 zVvlXwHh_|jz+t6v3@$R4Zx`ko8c5A?5UqGYms}oo#85(Hk}q33U=n8-Hio#{)Pj}v zB5P4QsN|2`#m(_$8RTtU7;!XTX_<6i^Q^%xYRclS-KHd`W_e&6bzCY)bxX|8PQ7{y z6{x<;_SP}5&_+@tjzgW!&@l(|EEH@F1I#pP)`Mv&O?MZ;^l8MB%X2`@eOI5sP&AzJ z@F|BE#3&jlJVaX6LLMfkDdft)3`rA)U6S43-yjeytm{!`zLvj=!np`Rz8WC0y@P;j z-~H&TnIGHk1^+~4z?{h>aC6~_H?rKh(1?ZctUaEy=pjt8CjLTMIDk7u)~$VpJ~$p$ zy^NnI)Qc3g@L>O8wl$aCsnUheM+ixH8D!f|$5Gul9KFwMCcjk^u z0sZG9>Mr7eT{OO0)A;0+t%#)W(7nLX;i-B)q8l3F>?%50IdT)Vv9Ps_Lg@yIq6}Q4i8q8cD7^6h|~ht)s&~Q*2*Bt z9ehGQohkMcOQ-x2p22%jer?{MIK>5}eqY=V3vp<7i@G(_w*;iG_$A~pH}_~tnOrM* zA~MsLPCZ_}tgKF(!(iP7Y|yQ%!+3d`#6Oj*`iIq&<)L}V-VEyE7Yyli1T5)(*0)XN zgG0ktn3*uSi1uX1Pwb58orI6|iiKT*HI)bMyYntGz9WiOA9-p=f7H%GX$K`X*UkP& z)8hr z!7Dsg$agW5(EOPjACqoJMJOiMU%rDo^B9`O)E^9{%zmG+gfW03D}a8d%(kI>fq{Li zFi_Hs@hF=vAj{(0IkKt&Cumj5M32+aOt7OyB7rUWY(}t*lRhiClhuU=+GmTg%FhLE z*4#6nXmAW;-I7#ZEal1%Yaf}LF;CJ(nY_3!bVqud9GB)xCi7Lti@99~z5#A<$F4@WOH%ysRQAYd z(?rjSwd`W6FEdb0j%eO}-NREmm3tRv82VVop~o_ zXf`YQ2AN^5 zxlj^+oKw8DmoILUzT-BLg7=^ zX7xu4skHvKgEH9i>-dtmM(YWywAe->=icMRcb7efZT1LSW(Z)8SeRBnP!BG{CDV_g zId?fM7R7~@1a)eHpf_H~u`d!MDR%AV(xtu*%Ci#YJ%VdYHFnzmHS58{L( zP%!yjn}tP6pqk`;@d=WAK%x4e*bC)DyA9bBkbE#X)?Mf7MTYcF+RSqEFpoH2 zU5EPxQR(*CBt`2qrfFYLc%wFJ5WgG?TI>W`p7N#IfRHWirY~1Q`A$C8>A{b>#N&S| z)!xUSF)3L*wmTle|Fp)yel;X{%jtZAuHZpxLbgi!$$cmh=XQzhd$-U1q8TeVcS@^iqMgA8Q-5(B#|&lpRYd55L$y!`OO)4{-N0mDjWmi#l3bfMQ_ zZv`k!o3!opx}`}iuMec2yUa;wD4gE1i;J z=;B>&^lX+BKX$J$m-u&CTH|~wvW10{305T9?CwB}U}mF%jA9FlP3+2d1*l6rOoR`O z$EuWWS*TMqOqK_&OzpvjxKi(9RdvQ9QUtQC?yFOC)aV7T5OuT((;_3G0LIl1FgFmzlChwjQ-)E-6c(|7ZXG45lo()!X@V z1NHakkAu-~<%8dq*T1Iny(z0I(T~3hM9olnu|w32#V5RVq|Ur;vk=QJj{H{Hx0rsq z!(k!Cv@(sHK1!E8W$$P|a|m+MT1Yk%kYpGH;>ix$p99G$Js-ByHZWsTSnt%Bbr!_f z^=Rs2K}kX%*d2^d;DD&6v?>;d5VI$#53CVOU?hzuNnkcM25O>bq@oGw@3E7~n2*sD zOqV#(Dc}#)K(;tF4SrQxla(Jygl?Bk4r1{>05yHcnc)6U~+Z4J*^oMV$w-il~n z%s_*VRdz9kmB9_Pk|TZn4k|UypV}QHd+SDgU9O8WP>VPb8FA!xs^b)>Q`ZU|v|=Wz zmbGcJ+l@N?om(xL@g}lBilsuWCs6dX>FTR-mcdY_L&IPv@xTA5|IBX|OMUNSd=+Tf zD1fPp?QzZU8|Ro0Xsa9;#KKxJfxNgm zPy{J)){=E+@?5yjFKD^XbhwS)73&|=!UDee2&nU^N@4&9Cp6c5sC?AYXR~eGBhl$} zvc>7#gLy?R@|W2GKA4Uhp$RAFdD-kBKJlGQ#87l)WhCl10ch63KS_Auf-*GRdz}K} zxDO)A*i;=SoR8+&bmVJK*7+T3UF?)+($2XPie`uDhoGcy998K+w^bCJPXgZgm^`;I)4m!J` zV%AxLm%0MP3hWeNgEaE3Hg3fbB)_%Gm1+neW*h5Rk47PrX(5IhQQi9rFS*ApdPa0W zERP(G1-!~Hqa=LBKl5^gi=%l{3E?(k!KU&fWxUDGAxf@7EWoe3D27pawt}@29VK76 zf2svBdDvHcv(whhBW|KBrXCoTP)x3c19epRW)iY+3}C6i4%QN@TqRdHrC7mXp&v_| zo6p3#3y;Ht4^Fb(+~Cyqm=)TI&`Y}&Y2rzaD)d9&0B7CEMo6MM(6&00)h94i7_|>s zNCIM{&Gn9<%dM5MpRuut7s7dhfes+*x>POsQMp@a8(`06AWsK`>U z3o88dQ>)~HASp}u$Z(wto^r#BG*`z`X>muugem2t*DA~W8OO{Gvg{S*yM@FkEYpa& zLNGxtn85sFp#^-e;6Y#zA#`X#HsH!(Wz~gi_8x=)Q+g+$IKJ)AN4z}DXEeKvmpKnP z#w%Pr53x}YS)jMt}?h|KNuClhwc6_&m z!s$d^MQe-9E-pY=>VieVvm{9WyzkOfzU`BGXZ{f8N*?$;9hW1pZ5FnTn^_CnzX3w0L!JyK$I8ssHl{Gi_>BXQ zN+UoWa#8YfjSEJap}8A)pC5_Uqe@tJHo6S%fRVeDKPizq2czW?P1oXQPZFs0FIFJV zcau)!$BG1#QrYbpTY!DGr^aWKWr^7r|Eh%G8T+^O5g}% z7G(>%o7_44Lak8|vt!(@PK#Pr80_<%3$5|fi3Q&Yb>2DZHOWTdz8n^u8w}`K*n?b} z#VgHTQ~|tgMZjztxc|ben!B3Hy@tl7<-ASug^v!`9V>l{d@`E5zGH-!h`F0po-@OT zbSx*b`YdNi3>jm8erav=2v+Yq&o?*x14E;`@R70Nkyp_7R~TZL>X^eft$gI$Q6m10 zBgFW>93pACR_h*m=+zsP05Xq@*)a!W8)lwu6w+ONn2s{A z7otz1Y+d83XtxMrhppv@4=3^s&6cVcAaFdD?o#g~88KH}Dye~Q(pXAt4Eh1#y)Fd; zD*Pd*ruGp^3GXv=S44_(W5{m}PojV7gi)g{+A;rT9)B;u zts;*-D3@u1}fR+YCgbn?e+4M|AYq!_vrPE2dd*8cKjB_zd0IjVO*9$+# zokA)eE{gEmB^k8}-1#CiRepw9kr$=K+d2fdw;ehPOI@X}5gGk>8E&U+OW2 zlehWLhCA=B^UY3&nvR}75P#qid0^?5b>QDt`0$A(qrj;tX^L8St`U%BtkIOA@{c(C zx4DBORnOvEC-a*p!<(Vwf6b4xk-dYNjrFe~_53kb*~t|O01)I4hXs&7Y#elS{`{Vf z4n8+N89glrEdv>(oFpI+F7Uqz-oE&+7bpPwpP%6YPWb=$_{Y`e%|!wf0GNaz`acQ& z6Do-_AmYt;@@<@d7*GBa>TNIpdA_{=1@+EL;(tQ@;}1Xp=&@6X%@1Qnwu1ryz+(O< zYSMq9{vignwXy%FYsBxNUOYy)g1^Z_^xvKlVc(54>#Y>}kH2kTV`Z!Nf5BAznHw`=1>$$3_Pz7TufzQM9`OGocHcXm{EB_{9{X?oPyVL~`Ssgf_pcOaPJgBNSNij>4F306 z-fO~sWzqEdE6Y2D*zXa%7oPk|pz8Zqg1^X6evjn+y5FxPJV}2gdA|zydkpVagnnfR zO8G0p`}LvUV|ZVp`;|c@`>zb|3U$MY z^Y4v(LYn$-(z~8?)}OHT>e+4Uz5N83IE@B nhxh5yulPh2f8zg|JpFf|pl>+<0073@o9OM%?f3Rm0D%7o(m9di literal 0 HcmV?d00001 diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index 0f1f005ce3edf..ec3ad9933cf60 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -19,7 +19,6 @@ import os import sys -from threading import RLock, Timer from py4j.java_gateway import java_import, JavaObject @@ -33,63 +32,6 @@ __all__ = ["StreamingContext"] -class Py4jCallbackConnectionCleaner(object): - - """ - A cleaner to clean up callback connections that are not closed by Py4j. See SPARK-12617. - It will scan all callback connections every 30 seconds and close the dead connections. - """ - - def __init__(self, gateway): - self._gateway = gateway - self._stopped = False - self._timer = None - self._lock = RLock() - - def start(self): - if self._stopped: - return - - def clean_closed_connections(): - from py4j.java_gateway import quiet_close, quiet_shutdown - - callback_server = self._gateway._callback_server - if callback_server: - with callback_server.lock: - try: - closed_connections = [] - for connection in callback_server.connections: - if not connection.isAlive(): - quiet_close(connection.input) - quiet_shutdown(connection.socket) - quiet_close(connection.socket) - closed_connections.append(connection) - - for closed_connection in closed_connections: - callback_server.connections.remove(closed_connection) - except Exception: - import traceback - traceback.print_exc() - - self._start_timer(clean_closed_connections) - - self._start_timer(clean_closed_connections) - - def _start_timer(self, f): - with self._lock: - if not self._stopped: - self._timer = Timer(30.0, f) - self._timer.daemon = True - self._timer.start() - - def stop(self): - with self._lock: - self._stopped = True - if self._timer: - self._timer.cancel() - self._timer = None - - class StreamingContext(object): """ Main entry point for Spark Streaming functionality. A StreamingContext @@ -105,9 +47,6 @@ class StreamingContext(object): # Reference to a currently active StreamingContext _activeContext = None - # A cleaner to clean leak sockets of callback server every 30 seconds - _py4j_cleaner = None - def __init__(self, sparkContext, batchDuration=None, jssc=None): """ Create a new StreamingContext. @@ -155,34 +94,12 @@ def _ensure_initialized(cls): # 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) - _py4j_cleaner = Py4jCallbackConnectionCleaner(gw) - _py4j_cleaner.start() + jgws.resetCallbackClient(jgws.getCallbackClient().getAddress(), gw._python_proxy_port) # register serializer for TransformFunction # it happens before creating SparkContext when loading from checkpointing - if cls._transformerSerializer is None: - transformer_serializer = TransformFunctionSerializer() - transformer_serializer.init( - SparkContext._active_spark_context, CloudPickleSerializer(), gw) - # SPARK-12511 streaming driver with checkpointing unable to finalize leading to OOM - # There is an issue that Py4J's PythonProxyHandler.finalize blocks forever. - # (https://github.com/bartdag/py4j/pull/184) - # - # Py4j will create a PythonProxyHandler in Java for "transformer_serializer" when - # calling "registerSerializer". If we call "registerSerializer" twice, the second - # PythonProxyHandler will override the first one, then the first one will be GCed and - # trigger "PythonProxyHandler.finalize". To avoid that, we should not call - # "registerSerializer" more than once, so that "PythonProxyHandler" in Java side won't - # be GCed. - # - # TODO Once Py4J fixes this issue, we should upgrade Py4j to the latest version. - transformer_serializer.gateway.jvm.PythonDStream.registerSerializer( - transformer_serializer) - cls._transformerSerializer = transformer_serializer - else: - cls._transformerSerializer.init( - SparkContext._active_spark_context, CloudPickleSerializer(), gw) + cls._transformerSerializer = TransformFunctionSerializer( + SparkContext._active_spark_context, CloudPickleSerializer(), gw) @classmethod def getOrCreate(cls, checkpointPath, setupFunc): diff --git a/python/pyspark/streaming/util.py b/python/pyspark/streaming/util.py index e617fc9ce9eec..abbbf6eb9394f 100644 --- a/python/pyspark/streaming/util.py +++ b/python/pyspark/streaming/util.py @@ -89,10 +89,11 @@ class TransformFunctionSerializer(object): it uses this class to invoke Python, which returns the serialized function as a byte array. """ - def init(self, ctx, serializer, gateway=None): + 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) self.failure = None def dumps(self, id): diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index d8d9d00d64ebc..0c37985a670b2 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -27,4 +27,4 @@ fi 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.9-src.zip:${PYTHONPATH}" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.9.1-src.zip:${PYTHONPATH}" 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 953fe95177f02..8c9beccc2922c 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 @@ -169,16 +169,6 @@ private[python] object PythonDStream { 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 diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 8cf438be587dc..d4ca255953a48 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -1044,9 +1044,9 @@ private[spark] class Client( val pyArchivesFile = new File(pyLibPath, "pyspark.zip") require(pyArchivesFile.exists(), "pyspark.zip not found; cannot run pyspark application in YARN mode.") - val py4jFile = new File(pyLibPath, "py4j-0.9-src.zip") + val py4jFile = new File(pyLibPath, "py4j-0.9.1-src.zip") require(py4jFile.exists(), - "py4j-0.9-src.zip not found; cannot run pyspark application in YARN mode.") + "py4j-0.9.1-src.zip not found; cannot run pyspark application in YARN mode.") Seq(pyArchivesFile.getAbsolutePath(), py4jFile.getAbsolutePath()) } } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 6db012a77a936..b91c4be2ea875 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -151,9 +151,9 @@ class YarnClusterSuite extends BaseYarnClusterSuite { // When running tests, let's not assume the user has built the assembly module, which also // creates the pyspark archive. Instead, let's use PYSPARK_ARCHIVES_PATH to point at the // needed locations. - val sparkHome = sys.props("spark.test.home"); + val sparkHome = sys.props("spark.test.home") val pythonPath = Seq( - s"$sparkHome/python/lib/py4j-0.9-src.zip", + s"$sparkHome/python/lib/py4j-0.9.1-src.zip", s"$sparkHome/python") val extraEnv = Map( "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator), From 9247084962259ebbbac4c5a80a6ccb271776f019 Mon Sep 17 00:00:00 2001 From: Nong Li Date: Tue, 12 Jan 2016 18:21:04 -0800 Subject: [PATCH 086/110] [SPARK-12785][SQL] Add ColumnarBatch, an in memory columnar format for execution. There are many potential benefits of having an efficient in memory columnar format as an alternate to UnsafeRow. This patch introduces ColumnarBatch/ColumnarVector which starts this effort. The remaining implementation can be done as follow up patches. As stated in the in the JIRA, there are useful external components that operate on memory in a simple columnar format. ColumnarBatch would serve that purpose and could server as a zero-serialization/zero-copy exchange for this use case. This patch supports running the underlying data either on heap or off heap. On heap runs a bit faster but we would need offheap for zero-copy exchanges. Currently, this mode is hidden behind one interface (ColumnVector). This differs from Parquet or the existing columnar cache because this is *not* intended to be used as a storage format. The focus is entirely on CPU efficiency as we expect to only have 1 of these batches in memory per task. The layout of the values is just dense arrays of the value type. Author: Nong Li Author: Nong Closes #10628 from nongli/spark-12635. --- .../execution/vectorized/ColumnVector.java | 176 ++++++++++ .../execution/vectorized/ColumnarBatch.java | 296 ++++++++++++++++ .../vectorized/OffHeapColumnVector.java | 179 ++++++++++ .../vectorized/OnHeapColumnVector.java | 175 ++++++++++ .../vectorized/ColumnarBatchBenchmark.scala | 320 ++++++++++++++++++ .../vectorized/ColumnarBatchSuite.scala | 317 +++++++++++++++++ 6 files changed, 1463 insertions(+) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java new file mode 100644 index 0000000000000..d9dde92ceb6d7 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.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.sql.execution.vectorized; + +import org.apache.spark.sql.types.DataType; + +/** + * This class represents a column of values and provides the main APIs to access the data + * values. It supports all the types and contains get/put APIs as well as their batched versions. + * The batched versions are preferable whenever possible. + * + * Most of the APIs take the rowId as a parameter. This is the local 0-based row id for values + * in the current RowBatch. + * + * A ColumnVector should be considered immutable once originally created. In other words, it is not + * valid to call put APIs after reads until reset() is called. + */ +public abstract class ColumnVector { + /** + * Allocates a column with each element of size `width` either on or off heap. + */ + public static ColumnVector allocate(int capacity, DataType type, boolean offHeap) { + if (offHeap) { + return new OffHeapColumnVector(capacity, type); + } else { + return new OnHeapColumnVector(capacity, type); + } + } + + public final DataType dataType() { return type; } + + /** + * Resets this column for writing. The currently stored values are no longer accessible. + */ + public void reset() { + numNulls = 0; + if (anyNullsSet) { + putNotNulls(0, capacity); + anyNullsSet = false; + } + } + + /** + * Cleans up memory for this column. The column is not usable after this. + * TODO: this should probably have ref-counted semantics. + */ + public abstract void close(); + + /** + * Returns the number of nulls in this column. + */ + public final int numNulls() { return numNulls; } + + /** + * Returns true if any of the nulls indicator are set for this column. This can be used + * as an optimization to prevent setting nulls. + */ + public final boolean anyNullsSet() { return anyNullsSet; } + + /** + * Returns the off heap ptr for the arrays backing the NULLs and values buffer. Only valid + * to call for off heap columns. + */ + public abstract long nullsNativeAddress(); + public abstract long valuesNativeAddress(); + + /** + * Sets the value at rowId to null/not null. + */ + public abstract void putNotNull(int rowId); + public abstract void putNull(int rowId); + + /** + * Sets the values from [rowId, rowId + count) to null/not null. + */ + public abstract void putNulls(int rowId, int count); + public abstract void putNotNulls(int rowId, int count); + + /** + * Returns whether the value at rowId is NULL. + */ + public abstract boolean getIsNull(int rowId); + + /** + * Sets the value at rowId to `value`. + */ + public abstract void putInt(int rowId, int value); + + /** + * Sets values from [rowId, rowId + count) to value. + */ + public abstract void putInts(int rowId, int count, int value); + + /** + * Sets values from [rowId, rowId + count) to [src + srcIndex, src + srcIndex + count) + */ + public abstract void putInts(int rowId, int count, int[] src, int srcIndex); + + /** + * Sets values from [rowId, rowId + count) to [src + srcIndex, src + srcIndex + count) + * The data in src must be 4-byte little endian ints. + */ + public abstract void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex); + + /** + * Returns the integer for rowId. + */ + public abstract int getInt(int rowId); + + /** + * Sets the value at rowId to `value`. + */ + public abstract void putDouble(int rowId, double value); + + /** + * Sets values from [rowId, rowId + count) to value. + */ + public abstract void putDoubles(int rowId, int count, double value); + + /** + * Sets values from [rowId, rowId + count) to [src + srcIndex, src + srcIndex + count) + * src should contain `count` doubles written as ieee format. + */ + public abstract void putDoubles(int rowId, int count, double[] src, int srcIndex); + + /** + * Sets values from [rowId, rowId + count) to [src + srcIndex, src + srcIndex + count) + * The data in src must be ieee formated doubles. + */ + public abstract void putDoubles(int rowId, int count, byte[] src, int srcIndex); + + /** + * Returns the double for rowId. + */ + public abstract double getDouble(int rowId); + + /** + * Maximum number of rows that can be stored in this column. + */ + protected final int capacity; + + /** + * Number of nulls in this column. This is an optimization for the reader, to skip NULL checks. + */ + protected int numNulls; + + /** + * True if there is at least one NULL byte set. This is an optimization for the writer, to skip + * having to clear NULL bits. + */ + protected boolean anyNullsSet; + + /** + * Data type for this column. + */ + protected final DataType type; + + protected ColumnVector(int capacity, DataType type) { + this.capacity = capacity; + this.type = type; + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java new file mode 100644 index 0000000000000..47defac4534dc --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java @@ -0,0 +1,296 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.vectorized; + +import java.util.Arrays; +import java.util.Iterator; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.CalendarInterval; +import org.apache.spark.unsafe.types.UTF8String; + +import org.apache.commons.lang.NotImplementedException; + +/** + * This class is the in memory representation of rows as they are streamed through operators. It + * is designed to maximize CPU efficiency and not storage footprint. Since it is expected that + * each operator allocates one of thee objects, the storage footprint on the task is negligible. + * + * The layout is a columnar with values encoded in their native format. Each RowBatch contains + * a horizontal partitioning of the data, split into columns. + * + * The ColumnarBatch supports either on heap or offheap modes with (mostly) the identical API. + * + * TODO: + * - There are many TODOs for the existing APIs. They should throw a not implemented exception. + * - Compaction: The batch and columns should be able to compact based on a selection vector. + */ +public final class ColumnarBatch { + private static final int DEFAULT_BATCH_SIZE = 4 * 1024; + + private final StructType schema; + private final int capacity; + private int numRows; + private final ColumnVector[] columns; + + // True if the row is filtered. + private final boolean[] filteredRows; + + // Total number of rows that have been filtered. + private int numRowsFiltered = 0; + + public static ColumnarBatch allocate(StructType schema, boolean offHeap) { + return new ColumnarBatch(schema, DEFAULT_BATCH_SIZE, offHeap); + } + + public static ColumnarBatch allocate(StructType schema, boolean offHeap, int maxRows) { + return new ColumnarBatch(schema, maxRows, offHeap); + } + + /** + * Called to close all the columns in this batch. It is not valid to access the data after + * calling this. This must be called at the end to clean up memory allcoations. + */ + public void close() { + for (ColumnVector c: columns) { + c.close(); + } + } + + /** + * Adapter class to interop with existing components that expect internal row. A lot of + * performance is lost with this translation. + */ + public final class Row extends InternalRow { + private int rowId; + + /** + * Marks this row as being filtered out. This means a subsequent iteration over the rows + * in this batch will not include this row. + */ + public final void markFiltered() { + ColumnarBatch.this.markFiltered(rowId); + } + + @Override + public final int numFields() { + return ColumnarBatch.this.numCols(); + } + + @Override + public final InternalRow copy() { + throw new NotImplementedException(); + } + + @Override + public final boolean anyNull() { + throw new NotImplementedException(); + } + + @Override + public final boolean isNullAt(int ordinal) { + return ColumnarBatch.this.column(ordinal).getIsNull(rowId); + } + + @Override + public final boolean getBoolean(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final byte getByte(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final short getShort(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final int getInt(int ordinal) { + return ColumnarBatch.this.column(ordinal).getInt(rowId); + } + + @Override + public final long getLong(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final float getFloat(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final double getDouble(int ordinal) { + return ColumnarBatch.this.column(ordinal).getDouble(rowId); + } + + @Override + public final Decimal getDecimal(int ordinal, int precision, int scale) { + throw new NotImplementedException(); + } + + @Override + public final UTF8String getUTF8String(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final byte[] getBinary(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final CalendarInterval getInterval(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final InternalRow getStruct(int ordinal, int numFields) { + throw new NotImplementedException(); + } + + @Override + public final ArrayData getArray(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final MapData getMap(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final Object get(int ordinal, DataType dataType) { + throw new NotImplementedException(); + } + } + + /** + * Returns an iterator over the rows in this batch. This skips rows that are filtered out. + */ + public Iterator rowIterator() { + final int maxRows = ColumnarBatch.this.numRows(); + final Row row = new Row(); + return new Iterator() { + int rowId = 0; + + @Override + public boolean hasNext() { + while (rowId < maxRows && ColumnarBatch.this.filteredRows[rowId]) { + ++rowId; + } + return rowId < maxRows; + } + + @Override + public Row next() { + assert(hasNext()); + while (rowId < maxRows && ColumnarBatch.this.filteredRows[rowId]) { + ++rowId; + } + row.rowId = rowId++; + return row; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + + /** + * Resets the batch for writing. + */ + public void reset() { + for (int i = 0; i < numCols(); ++i) { + columns[i].reset(); + } + if (this.numRowsFiltered > 0) { + Arrays.fill(filteredRows, false); + } + this.numRows = 0; + this.numRowsFiltered = 0; + } + + /** + * Sets the number of rows that are valid. + */ + public void setNumRows(int numRows) { + assert(numRows <= this.capacity); + this.numRows = numRows; + } + + /** + * Returns the number of columns that make up this batch. + */ + public int numCols() { return columns.length; } + + /** + * Returns the number of rows for read, including filtered rows. + */ + public int numRows() { return numRows; } + + /** + * Returns the number of valid rowss. + */ + public int numValidRows() { + assert(numRowsFiltered <= numRows); + return numRows - numRowsFiltered; + } + + /** + * Returns the max capacity (in number of rows) for this batch. + */ + public int capacity() { return capacity; } + + /** + * Returns the column at `ordinal`. + */ + public ColumnVector column(int ordinal) { return columns[ordinal]; } + + /** + * Marks this row as being filtered out. This means a subsequent iteration over the rows + * in this batch will not include this row. + */ + public final void markFiltered(int rowId) { + assert(filteredRows[rowId] == false); + filteredRows[rowId] = true; + ++numRowsFiltered; + } + + private ColumnarBatch(StructType schema, int maxRows, boolean offHeap) { + this.schema = schema; + this.capacity = maxRows; + this.columns = new ColumnVector[schema.size()]; + this.filteredRows = new boolean[maxRows]; + + for (int i = 0; i < schema.fields().length; ++i) { + StructField field = schema.fields()[i]; + columns[i] = ColumnVector.allocate(maxRows, field.dataType(), offHeap); + } + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java new file mode 100644 index 0000000000000..2a9a2d1104b22 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.vectorized; + +import java.nio.ByteOrder; + +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DoubleType; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.unsafe.Platform; + + +import org.apache.commons.lang.NotImplementedException; + +/** + * Column data backed using offheap memory. + */ +public final class OffHeapColumnVector extends ColumnVector { + // The data stored in these two allocations need to maintain binary compatible. We can + // directly pass this buffer to external components. + private long nulls; + private long data; + + protected OffHeapColumnVector(int capacity, DataType type) { + super(capacity, type); + if (!ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN)) { + throw new NotImplementedException("Only little endian is supported."); + } + + this.nulls = Platform.allocateMemory(capacity); + if (type instanceof IntegerType) { + this.data = Platform.allocateMemory(capacity * 4); + } else if (type instanceof DoubleType) { + this.data = Platform.allocateMemory(capacity * 8); + } else { + throw new RuntimeException("Unhandled " + type); + } + reset(); + } + + @Override + public final long valuesNativeAddress() { + return data; + } + + @Override + public long nullsNativeAddress() { + return nulls; + } + + @Override + public final void close() { + Platform.freeMemory(nulls); + Platform.freeMemory(data); + nulls = 0; + data = 0; + } + + // + // APIs dealing with nulls + // + + @Override + public final void putNotNull(int rowId) { + Platform.putByte(null, nulls + rowId, (byte) 0); + } + + @Override + public final void putNull(int rowId) { + Platform.putByte(null, nulls + rowId, (byte) 1); + ++numNulls; + anyNullsSet = true; + } + + @Override + public final void putNulls(int rowId, int count) { + long offset = nulls + rowId; + for (int i = 0; i < count; ++i, ++offset) { + Platform.putByte(null, offset, (byte) 1); + } + anyNullsSet = true; + numNulls += count; + } + + @Override + public final void putNotNulls(int rowId, int count) { + long offset = nulls + rowId; + for (int i = 0; i < count; ++i, ++offset) { + Platform.putByte(null, offset, (byte) 0); + } + } + + @Override + public final boolean getIsNull(int rowId) { + return Platform.getByte(null, nulls + rowId) == 1; + } + + // + // APIs dealing with ints + // + + @Override + public final void putInt(int rowId, int value) { + Platform.putInt(null, data + 4 * rowId, value); + } + + @Override + public final void putInts(int rowId, int count, int value) { + long offset = data + 4 * rowId; + for (int i = 0; i < count; ++i, offset += 4) { + Platform.putInt(null, offset, value); + } + } + + @Override + public final void putInts(int rowId, int count, int[] src, int srcIndex) { + Platform.copyMemory(src, Platform.INT_ARRAY_OFFSET + srcIndex * 4, + null, data + 4 * rowId, count * 4); + } + + @Override + public final void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, + null, data + 4 * rowId, count * 4); + } + + @Override + public final int getInt(int rowId) { + return Platform.getInt(null, data + 4 * rowId); + } + + // + // APIs dealing with doubles + // + + @Override + public final void putDouble(int rowId, double value) { + Platform.putDouble(null, data + rowId * 8, value); + } + + @Override + public final void putDoubles(int rowId, int count, double value) { + long offset = data + 8 * rowId; + for (int i = 0; i < count; ++i, offset += 8) { + Platform.putDouble(null, offset, value); + } + } + + @Override + public final void putDoubles(int rowId, int count, double[] src, int srcIndex) { + Platform.copyMemory(src, Platform.DOUBLE_ARRAY_OFFSET + srcIndex * 8, + null, data + 8 * rowId, count * 8); + } + + @Override + public final void putDoubles(int rowId, int count, byte[] src, int srcIndex) { + Platform.copyMemory(src, Platform.DOUBLE_ARRAY_OFFSET + srcIndex, + null, data + rowId * 8, count * 8); + } + + @Override + public final double getDouble(int rowId) { + return Platform.getDouble(null, data + rowId * 8); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java new file mode 100644 index 0000000000000..a7b3addf11b14 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.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.sql.execution.vectorized; + +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DoubleType; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.unsafe.Platform; + +import java.nio.ByteBuffer; +import java.nio.DoubleBuffer; +import java.util.Arrays; + +/** + * A column backed by an in memory JVM array. This stores the NULLs as a byte per value + * and a java array for the values. + */ +public final class OnHeapColumnVector extends ColumnVector { + // The data stored in these arrays need to maintain binary compatible. We can + // directly pass this buffer to external components. + + // This is faster than a boolean array and we optimize this over memory footprint. + private byte[] nulls; + + // Array for each type. Only 1 is populated for any type. + private int[] intData; + private double[] doubleData; + + protected OnHeapColumnVector(int capacity, DataType type) { + super(capacity, type); + if (type instanceof IntegerType) { + this.intData = new int[capacity]; + } else if (type instanceof DoubleType) { + this.doubleData = new double[capacity]; + } else { + throw new RuntimeException("Unhandled " + type); + } + this.nulls = new byte[capacity]; + reset(); + } + + @Override + public final long valuesNativeAddress() { + throw new RuntimeException("Cannot get native address for on heap column"); + } + @Override + public final long nullsNativeAddress() { + throw new RuntimeException("Cannot get native address for on heap column"); + } + + @Override + public final void close() { + nulls = null; + intData = null; + doubleData = null; + } + + + // + // APIs dealing with nulls + // + + @Override + public final void putNotNull(int rowId) { + nulls[rowId] = (byte)0; + } + + @Override + public final void putNull(int rowId) { + nulls[rowId] = (byte)1; + ++numNulls; + anyNullsSet = true; + } + + @Override + public final void putNulls(int rowId, int count) { + for (int i = 0; i < count; ++i) { + nulls[rowId + i] = (byte)1; + } + anyNullsSet = true; + numNulls += count; + } + + @Override + public final void putNotNulls(int rowId, int count) { + for (int i = 0; i < count; ++i) { + nulls[rowId + i] = (byte)0; + } + } + + @Override + public final boolean getIsNull(int rowId) { + return nulls[rowId] == 1; + } + + // + // APIs dealing with Ints + // + + @Override + public final void putInt(int rowId, int value) { + intData[rowId] = value; + } + + @Override + public final void putInts(int rowId, int count, int value) { + for (int i = 0; i < count; ++i) { + intData[i + rowId] = value; + } + } + + @Override + public final void putInts(int rowId, int count, int[] src, int srcIndex) { + System.arraycopy(src, srcIndex, intData, rowId, count); + } + + @Override + public final void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; + for (int i = 0; i < count; ++i) { + intData[i + rowId] = Platform.getInt(src, srcOffset);; + srcIndex += 4; + srcOffset += 4; + } + } + + @Override + public final int getInt(int rowId) { + return intData[rowId]; + } + + // + // APIs dealing with doubles + // + + @Override + public final void putDouble(int rowId, double value) { + doubleData[rowId] = value; + } + + @Override + public final void putDoubles(int rowId, int count, double value) { + Arrays.fill(doubleData, rowId, rowId + count, value); + } + + @Override + public final void putDoubles(int rowId, int count, double[] src, int srcIndex) { + System.arraycopy(src, srcIndex, doubleData, rowId, count); + } + + @Override + public final void putDoubles(int rowId, int count, byte[] src, int srcIndex) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, doubleData, + Platform.DOUBLE_ARRAY_OFFSET + rowId * 8, count * 8); + } + + @Override + public final double getDouble(int rowId) { + return doubleData[rowId]; + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala new file mode 100644 index 0000000000000..e28153d12a354 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala @@ -0,0 +1,320 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.datasources.parquet + +import java.nio.ByteBuffer + +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.execution.vectorized.ColumnVector +import org.apache.spark.sql.types.IntegerType +import org.apache.spark.unsafe.Platform +import org.apache.spark.util.Benchmark +import org.apache.spark.util.collection.BitSet + +/** + * Benchmark to low level memory access using different ways to manage buffers. + */ +object ColumnarBatchBenchmark { + + // This benchmark reads and writes an array of ints. + // TODO: there is a big (2x) penalty for a random access API for off heap. + // Note: carefully if modifying this code. It's hard to reason about the JIT. + def intAccess(iters: Long): Unit = { + val count = 8 * 1000 + + // Accessing a java array. + val javaArray = { i: Int => + val data = new Array[Int](count) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + data(i) = i + i += 1 + } + i = 0 + while (i < count) { + sum += data(i) + i += 1 + } + } + } + + // Accessing ByteBuffers + val byteBufferUnsafe = { i: Int => + val data = ByteBuffer.allocate(count * 4) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + Platform.putInt(data.array(), Platform.BYTE_ARRAY_OFFSET + i * 4, i) + i += 1 + } + i = 0 + while (i < count) { + sum += Platform.getInt(data.array(), Platform.BYTE_ARRAY_OFFSET + i * 4) + i += 1 + } + } + } + + // Accessing offheap byte buffers + val directByteBuffer = { i: Int => + val data = ByteBuffer.allocateDirect(count * 4).asIntBuffer() + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + data.put(i) + i += 1 + } + data.rewind() + i = 0 + while (i < count) { + sum += data.get() + i += 1 + } + data.rewind() + } + } + + // Accessing ByteBuffer using the typed APIs + val byteBufferApi = { i: Int => + val data = ByteBuffer.allocate(count * 4) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + data.putInt(i) + i += 1 + } + data.rewind() + i = 0 + while (i < count) { + sum += data.getInt() + i += 1 + } + data.rewind() + } + } + + // Using unsafe memory + val unsafeBuffer = { i: Int => + val data: Long = Platform.allocateMemory(count * 4) + var sum = 0L + for (n <- 0L until iters) { + var ptr = data + var i = 0 + while (i < count) { + Platform.putInt(null, ptr, i) + ptr += 4 + i += 1 + } + ptr = data + i = 0 + while (i < count) { + sum += Platform.getInt(null, ptr) + ptr += 4 + i += 1 + } + } + } + + // Access through the column API with on heap memory + val columnOnHeap = { i: Int => + val col = ColumnVector.allocate(count, IntegerType, false) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + col.putInt(i, i) + i += 1 + } + i = 0 + while (i < count) { + sum += col.getInt(i) + i += 1 + } + } + col.close + } + + // Access through the column API with off heap memory + def columnOffHeap = { i: Int => { + val col = ColumnVector.allocate(count, IntegerType, true) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + col.putInt(i, i) + i += 1 + } + i = 0 + while (i < count) { + sum += col.getInt(i) + i += 1 + } + } + col.close + }} + + // Access by directly getting the buffer backing the column. + val columnOffheapDirect = { i: Int => + val col = ColumnVector.allocate(count, IntegerType, true) + var sum = 0L + for (n <- 0L until iters) { + var addr = col.valuesNativeAddress() + var i = 0 + while (i < count) { + Platform.putInt(null, addr, i) + addr += 4 + i += 1 + } + i = 0 + addr = col.valuesNativeAddress() + while (i < count) { + sum += Platform.getInt(null, addr) + addr += 4 + i += 1 + } + } + col.close + } + + // Access by going through a batch of unsafe rows. + val unsafeRowOnheap = { i: Int => + val buffer = new Array[Byte](count * 16) + var sum = 0L + for (n <- 0L until iters) { + val row = new UnsafeRow(1) + var i = 0 + while (i < count) { + row.pointTo(buffer, Platform.BYTE_ARRAY_OFFSET + i * 16, 16) + row.setInt(0, i) + i += 1 + } + i = 0 + while (i < count) { + row.pointTo(buffer, Platform.BYTE_ARRAY_OFFSET + i * 16, 16) + sum += row.getInt(0) + i += 1 + } + } + } + + // Access by going through a batch of unsafe rows. + val unsafeRowOffheap = { i: Int => + val buffer = Platform.allocateMemory(count * 16) + var sum = 0L + for (n <- 0L until iters) { + val row = new UnsafeRow(1) + var i = 0 + while (i < count) { + row.pointTo(null, buffer + i * 16, 16) + row.setInt(0, i) + i += 1 + } + i = 0 + while (i < count) { + row.pointTo(null, buffer + i * 16, 16) + sum += row.getInt(0) + i += 1 + } + } + Platform.freeMemory(buffer) + } + + /* + Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz + Int Read/Write: Avg Time(ms) Avg Rate(M/s) Relative Rate + ------------------------------------------------------------------------- + Java Array 248.8 1317.04 1.00 X + ByteBuffer Unsafe 435.6 752.25 0.57 X + ByteBuffer API 1752.0 187.03 0.14 X + DirectByteBuffer 595.4 550.35 0.42 X + Unsafe Buffer 235.2 1393.20 1.06 X + Column(on heap) 189.8 1726.45 1.31 X + Column(off heap) 408.4 802.35 0.61 X + Column(off heap direct) 237.6 1379.12 1.05 X + UnsafeRow (on heap) 414.6 790.35 0.60 X + UnsafeRow (off heap) 487.2 672.58 0.51 X + */ + val benchmark = new Benchmark("Int Read/Write", count * iters) + benchmark.addCase("Java Array")(javaArray) + benchmark.addCase("ByteBuffer Unsafe")(byteBufferUnsafe) + benchmark.addCase("ByteBuffer API")(byteBufferApi) + benchmark.addCase("DirectByteBuffer")(directByteBuffer) + benchmark.addCase("Unsafe Buffer")(unsafeBuffer) + benchmark.addCase("Column(on heap)")(columnOnHeap) + benchmark.addCase("Column(off heap)")(columnOffHeap) + benchmark.addCase("Column(off heap direct)")(columnOffheapDirect) + benchmark.addCase("UnsafeRow (on heap)")(unsafeRowOnheap) + benchmark.addCase("UnsafeRow (off heap)")(unsafeRowOffheap) + benchmark.run() + } + + def booleanAccess(iters: Int): Unit = { + val count = 8 * 1024 + val benchmark = new Benchmark("Boolean Read/Write", iters * count) + benchmark.addCase("Bitset") { i: Int => { + val b = new BitSet(count) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + if (i % 2 == 0) b.set(i) + i += 1 + } + i = 0 + while (i < count) { + if (b.get(i)) sum += 1 + i += 1 + } + } + }} + + benchmark.addCase("Byte Array") { i: Int => { + val b = new Array[Byte](count) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + if (i % 2 == 0) b(i) = 1; + i += 1 + } + i = 0 + while (i < count) { + if (b(i) == 1) sum += 1 + i += 1 + } + } + }} + /* + Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz + Boolean Read/Write: Avg Time(ms) Avg Rate(M/s) Relative Rate + ------------------------------------------------------------------------- + Bitset 895.88 374.54 1.00 X + Byte Array 578.96 579.56 1.55 X + */ + benchmark.run() + } + + def main(args: Array[String]): Unit = { + intAccess(1024 * 40) + booleanAccess(1024 * 40) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala new file mode 100644 index 0000000000000..305a83e3e45c9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.vectorized + +import scala.collection.mutable +import scala.util.Random + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} +import org.apache.spark.unsafe.Platform + +class ColumnarBatchSuite extends SparkFunSuite { + test("Null Apis") { + (false :: true :: Nil).foreach { offHeap => { + val reference = mutable.ArrayBuffer.empty[Boolean] + + val column = ColumnVector.allocate(1024, IntegerType, offHeap) + var idx = 0 + assert(column.anyNullsSet() == false) + + column.putNotNull(idx) + reference += false + idx += 1 + assert(column.anyNullsSet() == false) + + column.putNull(idx) + reference += true + idx += 1 + assert(column.anyNullsSet() == true) + assert(column.numNulls() == 1) + + column.putNulls(idx, 3) + reference += true + reference += true + reference += true + idx += 3 + assert(column.anyNullsSet() == true) + + column.putNotNulls(idx, 4) + reference += false + reference += false + reference += false + reference += false + idx += 4 + assert(column.anyNullsSet() == true) + assert(column.numNulls() == 4) + + reference.zipWithIndex.foreach { v => + assert(v._1 == column.getIsNull(v._2)) + if (offHeap) { + val addr = column.nullsNativeAddress() + assert(v._1 == (Platform.getByte(null, addr + v._2) == 1), "index=" + v._2) + } + } + column.close + }} + } + + test("Int Apis") { + (false :: true :: Nil).foreach { offHeap => { + val seed = System.currentTimeMillis() + val random = new Random(seed) + val reference = mutable.ArrayBuffer.empty[Int] + + val column = ColumnVector.allocate(1024, IntegerType, offHeap) + var idx = 0 + + val values = (1 :: 2 :: 3 :: 4 :: 5 :: Nil).toArray + column.putInts(idx, 2, values, 0) + reference += 1 + reference += 2 + idx += 2 + + column.putInts(idx, 3, values, 2) + reference += 3 + reference += 4 + reference += 5 + idx += 3 + + val littleEndian = new Array[Byte](8) + littleEndian(0) = 7 + littleEndian(1) = 1 + littleEndian(4) = 6 + littleEndian(6) = 1 + + column.putIntsLittleEndian(idx, 1, littleEndian, 4) + column.putIntsLittleEndian(idx + 1, 1, littleEndian, 0) + reference += 6 + (1 << 16) + reference += 7 + (1 << 8) + idx += 2 + + column.putIntsLittleEndian(idx, 2, littleEndian, 0) + reference += 7 + (1 << 8) + reference += 6 + (1 << 16) + idx += 2 + + while (idx < column.capacity) { + val single = random.nextBoolean() + if (single) { + val v = random.nextInt() + column.putInt(idx, v) + reference += v + idx += 1 + } else { + val n = math.min(random.nextInt(column.capacity / 20), column.capacity - idx) + column.putInts(idx, n, n + 1) + var i = 0 + while (i < n) { + reference += (n + 1) + i += 1 + } + idx += n + } + } + + reference.zipWithIndex.foreach { v => + assert(v._1 == column.getInt(v._2), "Seed = " + seed + " Off Heap=" + offHeap) + if (offHeap) { + val addr = column.valuesNativeAddress() + assert(v._1 == Platform.getInt(null, addr + 4 * v._2)) + } + } + column.close + }} + } + + test("Double APIs") { + (false :: true :: Nil).foreach { offHeap => { + val seed = System.currentTimeMillis() + val random = new Random(seed) + val reference = mutable.ArrayBuffer.empty[Double] + + val column = ColumnVector.allocate(1024, DoubleType, offHeap) + var idx = 0 + + val values = (1.0 :: 2.0 :: 3.0 :: 4.0 :: 5.0 :: Nil).toArray + column.putDoubles(idx, 2, values, 0) + reference += 1.0 + reference += 2.0 + idx += 2 + + column.putDoubles(idx, 3, values, 2) + reference += 3.0 + reference += 4.0 + reference += 5.0 + idx += 3 + + val buffer = new Array[Byte](16) + Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET, 2.234) + Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET + 8, 1.123) + + column.putDoubles(idx, 1, buffer, 8) + column.putDoubles(idx + 1, 1, buffer, 0) + reference += 1.123 + reference += 2.234 + idx += 2 + + column.putDoubles(idx, 2, buffer, 0) + reference += 2.234 + reference += 1.123 + idx += 2 + + while (idx < column.capacity) { + val single = random.nextBoolean() + if (single) { + val v = random.nextDouble() + column.putDouble(idx, v) + reference += v + idx += 1 + } else { + val n = math.min(random.nextInt(column.capacity / 20), column.capacity - idx) + val v = random.nextDouble() + column.putDoubles(idx, n, v) + var i = 0 + while (i < n) { + reference += v + i += 1 + } + idx += n + } + } + + reference.zipWithIndex.foreach { v => + assert(v._1 == column.getDouble(v._2), "Seed = " + seed + " Off Heap=" + offHeap) + if (offHeap) { + val addr = column.valuesNativeAddress() + assert(v._1 == Platform.getDouble(null, addr + 8 * v._2)) + } + } + column.close + }} + } + + test("ColumnarBatch basic") { + (false :: true :: Nil).foreach { offHeap => { + val schema = new StructType() + .add("intCol", IntegerType) + .add("doubleCol", DoubleType) + .add("intCol2", IntegerType) + + val batch = ColumnarBatch.allocate(schema, offHeap) + assert(batch.numCols() == 3) + assert(batch.numRows() == 0) + assert(batch.numValidRows() == 0) + assert(batch.capacity() > 0) + assert(batch.rowIterator().hasNext == false) + + // Add a row [1, 1.1, NULL] + batch.column(0).putInt(0, 1) + batch.column(1).putDouble(0, 1.1) + batch.column(2).putNull(0) + batch.setNumRows(1) + + // Verify the results of the row. + assert(batch.numCols() == 3) + assert(batch.numRows() == 1) + assert(batch.numValidRows() == 1) + assert(batch.rowIterator().hasNext == true) + assert(batch.rowIterator().hasNext == true) + + assert(batch.column(0).getInt(0) == 1) + assert(batch.column(0).getIsNull(0) == false) + assert(batch.column(1).getDouble(0) == 1.1) + assert(batch.column(1).getIsNull(0) == false) + assert(batch.column(2).getIsNull(0) == true) + + // Verify the iterator works correctly. + val it = batch.rowIterator() + assert(it.hasNext()) + val row = it.next() + assert(row.getInt(0) == 1) + assert(row.isNullAt(0) == false) + assert(row.getDouble(1) == 1.1) + assert(row.isNullAt(1) == false) + assert(row.isNullAt(2) == true) + assert(it.hasNext == false) + assert(it.hasNext == false) + + // Filter out the row. + row.markFiltered() + assert(batch.numRows() == 1) + assert(batch.numValidRows() == 0) + assert(batch.rowIterator().hasNext == false) + + // Reset and add 3 throws + batch.reset() + assert(batch.numRows() == 0) + assert(batch.numValidRows() == 0) + assert(batch.rowIterator().hasNext == false) + + // Add rows [NULL, 2.2, 2], [3, NULL, 3], [4, 4.4, 4] + batch.column(0).putNull(0) + batch.column(1).putDouble(0, 2.2) + batch.column(2).putInt(0, 2) + + batch.column(0).putInt(1, 3) + batch.column(1).putNull(1) + batch.column(2).putInt(1, 3) + + batch.column(0).putInt(2, 4) + batch.column(1).putDouble(2, 4.4) + batch.column(2).putInt(2, 4) + batch.setNumRows(3) + + def rowEquals(x: InternalRow, y: Row): Unit = { + assert(x.isNullAt(0) == y.isNullAt(0)) + if (!x.isNullAt(0)) assert(x.getInt(0) == y.getInt(0)) + + assert(x.isNullAt(1) == y.isNullAt(1)) + if (!x.isNullAt(1)) assert(x.getDouble(1) == y.getDouble(1)) + + assert(x.isNullAt(2) == y.isNullAt(2)) + if (!x.isNullAt(2)) assert(x.getInt(2) == y.getInt(2)) + } + // Verify + assert(batch.numRows() == 3) + assert(batch.numValidRows() == 3) + val it2 = batch.rowIterator() + rowEquals(it2.next(), Row(null, 2.2, 2)) + rowEquals(it2.next(), Row(3, null, 3)) + rowEquals(it2.next(), Row(4, 4.4, 4)) + assert(!it.hasNext) + + // Filter out some rows and verify + batch.markFiltered(1) + assert(batch.numValidRows() == 2) + val it3 = batch.rowIterator() + rowEquals(it3.next(), Row(null, 2.2, 2)) + rowEquals(it3.next(), Row(4, 4.4, 4)) + assert(!it.hasNext) + + batch.markFiltered(2) + assert(batch.numValidRows() == 1) + val it4 = batch.rowIterator() + rowEquals(it4.next(), Row(null, 2.2, 2)) + + batch.close + }} + } +} From b3b9ad23cffc1c6d83168487093e4c03d49e1c2c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 12 Jan 2016 18:45:55 -0800 Subject: [PATCH 087/110] [SPARK-12788][SQL] Simplify BooleanEquality by using casts. Author: Reynold Xin Closes #10730 from rxin/SPARK-12788. --- .../catalyst/analysis/HiveTypeCoercion.scala | 30 ++++--------------- .../analysis/HiveTypeCoercionSuite.scala | 28 ++++++++++++++++- 2 files changed, 32 insertions(+), 26 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 e9e20670817fe..980b5d52fa8f7 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 @@ -482,27 +482,6 @@ object HiveTypeCoercion { private val trueValues = Seq(1.toByte, 1.toShort, 1, 1L, Decimal.ONE) private val falseValues = Seq(0.toByte, 0.toShort, 0, 0L, Decimal.ZERO) - private def buildCaseKeyWhen(booleanExpr: Expression, numericExpr: Expression) = { - CaseKeyWhen(numericExpr, Seq( - Literal(trueValues.head), booleanExpr, - Literal(falseValues.head), Not(booleanExpr), - Literal(false))) - } - - private def transform(booleanExpr: Expression, numericExpr: Expression) = { - If(Or(IsNull(booleanExpr), IsNull(numericExpr)), - Literal.create(null, BooleanType), - buildCaseKeyWhen(booleanExpr, numericExpr)) - } - - private def transformNullSafe(booleanExpr: Expression, numericExpr: Expression) = { - CaseWhen(Seq( - And(IsNull(booleanExpr), IsNull(numericExpr)), Literal(true), - Or(IsNull(booleanExpr), IsNull(numericExpr)), Literal(false), - buildCaseKeyWhen(booleanExpr, numericExpr) - )) - } - def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e @@ -511,6 +490,7 @@ object HiveTypeCoercion { // all other cases are considered as false. // We may simplify the expression if one side is literal numeric values + // TODO: Maybe these rules should go into the optimizer. case EqualTo(bool @ BooleanType(), Literal(value, _: NumericType)) if trueValues.contains(value) => bool case EqualTo(bool @ BooleanType(), Literal(value, _: NumericType)) @@ -529,13 +509,13 @@ object HiveTypeCoercion { if falseValues.contains(value) => And(IsNotNull(bool), Not(bool)) case EqualTo(left @ BooleanType(), right @ NumericType()) => - transform(left , right) + EqualTo(Cast(left, right.dataType), right) case EqualTo(left @ NumericType(), right @ BooleanType()) => - transform(right, left) + EqualTo(left, Cast(right, left.dataType)) case EqualNullSafe(left @ BooleanType(), right @ NumericType()) => - transformNullSafe(left, right) + EqualNullSafe(Cast(left, right.dataType), right) case EqualNullSafe(left @ NumericType(), right @ BooleanType()) => - transformNullSafe(right, left) + EqualNullSafe(left, Cast(right, left.dataType)) } } 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 23b11af9ac087..40378c6727667 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 @@ -320,7 +320,33 @@ class HiveTypeCoercionSuite extends PlanTest { ) } - test("type coercion simplification for equal to") { + test("BooleanEquality type cast") { + val be = HiveTypeCoercion.BooleanEquality + // Use something more than a literal to avoid triggering the simplification rules. + val one = Add(Literal(Decimal(1)), Literal(Decimal(0))) + + ruleTest(be, + EqualTo(Literal(true), one), + EqualTo(Cast(Literal(true), one.dataType), one) + ) + + ruleTest(be, + EqualTo(one, Literal(true)), + EqualTo(one, Cast(Literal(true), one.dataType)) + ) + + ruleTest(be, + EqualNullSafe(Literal(true), one), + EqualNullSafe(Cast(Literal(true), one.dataType), one) + ) + + ruleTest(be, + EqualNullSafe(one, Literal(true)), + EqualNullSafe(one, Cast(Literal(true), one.dataType)) + ) + } + + test("BooleanEquality simplification") { val be = HiveTypeCoercion.BooleanEquality ruleTest(be, From f14922cff84b1e0984ba4597d764615184126bdc Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 12 Jan 2016 19:24:50 -0800 Subject: [PATCH 088/110] [SPARK-12692][BUILD][CORE] Scala style: Fix the style violation (Space before ",") Fix the style violation (space before , and :). This PR is a followup for #10643 Author: Kousuke Saruta Closes #10719 from sarutak/SPARK-12692-followup-core. --- core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- core/src/main/scala/org/apache/spark/status/api/v1/api.scala | 2 +- core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala | 2 +- .../scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala | 2 +- scalastyle-config.xml | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) 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 18e8cddbc40db..57108dcedcf0c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala @@ -50,7 +50,7 @@ class CartesianRDD[T: ClassTag, U: ClassTag]( sc: SparkContext, var rdd1 : RDD[T], var rdd2 : RDD[U]) - extends RDD[Pair[T, U]](sc, Nil) + extends RDD[(T, U)](sc, Nil) with Serializable { val numPartitionsInRdd2 = rdd2.partitions.length 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 53e01a0dbfc06..9dad7944144d8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -95,7 +95,7 @@ abstract class RDD[T: ClassTag]( /** Construct an RDD with just a one-to-one dependency on one parent */ def this(@transient oneParent: RDD[_]) = - this(oneParent.context , List(new OneToOneDependency(oneParent))) + this(oneParent.context, List(new OneToOneDependency(oneParent))) private[spark] def conf = sc.conf // ======================================================================= diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 5feb1dc2e5b74..9cd52d6c2bef5 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -115,7 +115,7 @@ class StageData private[spark]( val status: StageStatus, val stageId: Int, val attemptId: Int, - val numActiveTasks: Int , + val numActiveTasks: Int, val numCompleteTasks: Int, val numFailedTasks: Int, 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 4e72b89bfcc40..76451788d2406 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -178,7 +178,7 @@ class DoubleRDDSuite extends SparkFunSuite with SharedSparkContext { test("WorksWithOutOfRangeWithInfiniteBuckets") { // Verify that out of range works with two buckets val rdd = sc.parallelize(Seq(10.01, -0.01, Double.NaN)) - val buckets = Array(-1.0/0.0 , 0.0, 1.0/0.0) + val buckets = Array(-1.0/0.0, 0.0, 1.0/0.0) val histogramResults = rdd.histogram(buckets) val expectedHistogramResults = Array(1, 1) assert(histogramResults === expectedHistogramResults) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala index 504e5780f3d8a..e111e2e9f6163 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala @@ -76,7 +76,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi test("check spark-class location correctly") { val conf = new SparkConf - conf.set("spark.mesos.executor.home" , "/mesos-home") + conf.set("spark.mesos.executor.home", "/mesos-home") val listenerBus = mock[LiveListenerBus] listenerBus.post( diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 2439a1f715aba..bc209ee6aa873 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -220,7 +220,7 @@ This file is divided into 3 sections: - COLON, COMMA + COMMA From dc7b3870fcfc2723319dbb8c53d721211a8116be Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Tue, 12 Jan 2016 21:41:38 -0800 Subject: [PATCH 089/110] [SPARK-12558][SQL] AnalysisException when multiple functions applied in GROUP BY clause cloud-fan Can you please take a look ? In this case, we are failing during check analysis while validating the aggregation expression. I have added a semanticEquals for HiveGenericUDF to fix this. Please let me know if this is the right way to address this issue. Author: Dilip Biswal Closes #10520 from dilipbiswal/spark-12558. --- .../org/apache/spark/sql/hive/HiveShim.scala | 23 +++++++++++++++++++ .../sql/hive/execution/HiveUDFSuite.scala | 7 ++++++ 2 files changed, 30 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala index b8cced0b80969..087b0c087c111 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala @@ -26,11 +26,13 @@ import scala.reflect.ClassTag import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.io.{Input, Output} +import com.google.common.base.Objects import org.apache.avro.Schema import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.exec.{UDF, Utilities} import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro import org.apache.hadoop.hive.serde2.ColumnProjectionUtils import org.apache.hadoop.hive.serde2.avro.{AvroGenericRecordWritable, AvroSerdeUtils} import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector @@ -45,6 +47,7 @@ private[hive] object HiveShim { // scale Hive 0.13 infers for BigDecimals from sources that don't specify them (e.g. UDFs) val UNLIMITED_DECIMAL_PRECISION = 38 val UNLIMITED_DECIMAL_SCALE = 18 + val HIVE_GENERIC_UDF_MACRO_CLS = "org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro" /* * This function in hive-0.13 become private, but we have to do this to walkaround hive bug @@ -123,6 +126,26 @@ private[hive] object HiveShim { // for Serialization def this() = this(null) + override def hashCode(): Int = { + if (functionClassName == HIVE_GENERIC_UDF_MACRO_CLS) { + Objects.hashCode(functionClassName, instance.asInstanceOf[GenericUDFMacro].getBody()) + } else { + functionClassName.hashCode() + } + } + + override def equals(other: Any): Boolean = other match { + case a: HiveFunctionWrapper if functionClassName == a.functionClassName => + // In case of udf macro, check to make sure they point to the same underlying UDF + if (functionClassName == HIVE_GENERIC_UDF_MACRO_CLS) { + a.instance.asInstanceOf[GenericUDFMacro].getBody() == + instance.asInstanceOf[GenericUDFMacro].getBody() + } else { + true + } + case _ => false + } + @transient def deserializeObjectByKryo[T: ClassTag]( kryo: Kryo, 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 c5ff8825abd7f..dfe33ba8b0502 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 @@ -350,6 +350,13 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { sqlContext.dropTempTable("testUDF") } + test("Hive UDF in group by") { + Seq(Tuple1(1451400761)).toDF("test_date").registerTempTable("tab1") + val count = sql("select date(cast(test_date as timestamp))" + + " from tab1 group by date(cast(test_date as timestamp))").count() + assert(count == 1) + } + test("SPARK-11522 select input_file_name from non-parquet table"){ withTempDir { tempDir => From cb7b864a24db4826e2942c186afe3cb8bd788b03 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 12 Jan 2016 22:25:20 -0800 Subject: [PATCH 090/110] [SPARK-12692][BUILD][SQL] Scala style: Fix the style violation (Space before ",") Fix the style violation (space before , and :). This PR is a followup for #10643 and rework of #10685 . Author: Kousuke Saruta Closes #10732 from sarutak/SPARK-12692-followup-sql. --- .../apache/spark/sql/catalyst/SqlParser.scala | 2 +- .../sql/catalyst/util/NumberConverter.scala | 2 +- .../BooleanSimplificationSuite.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../apache/spark/sql/execution/Exchange.scala | 2 +- .../apache/spark/sql/DatasetCacheSuite.scala | 2 +- .../org/apache/spark/sql/DatasetSuite.scala | 24 +++++++++---------- .../datasources/json/JsonSuite.scala | 2 +- .../hive/thriftserver/SparkSQLCLIDriver.scala | 2 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 4 ++-- 10 files changed, 22 insertions(+), 22 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 2a132d8b82bef..6ec408a673c79 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 @@ -203,7 +203,7 @@ object SqlParser extends AbstractSparkSQLParser with DataTypeParser { ) protected lazy val ordering: Parser[Seq[SortOrder]] = - ( rep1sep(expression ~ direction.? , ",") ^^ { + ( rep1sep(expression ~ direction.?, ",") ^^ { case exps => exps.map(pair => SortOrder(pair._1, pair._2.getOrElse(Ascending))) } ) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala index 9fefc5656aac0..e4417e0955143 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala @@ -122,7 +122,7 @@ object NumberConverter { * unsigned, otherwise it is signed. * NB: This logic is borrowed from org.apache.hadoop.hive.ql.ud.UDFConv */ - def convert(n: Array[Byte] , fromBase: Int, toBase: Int ): UTF8String = { + def convert(n: Array[Byte], fromBase: Int, toBase: Int ): UTF8String = { if (fromBase < Character.MIN_RADIX || fromBase > Character.MAX_RADIX || Math.abs(toBase) < Character.MIN_RADIX || Math.abs(toBase) > Character.MAX_RADIX) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index 000a3b7ecb7c6..6932f185b9d62 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -80,7 +80,7 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { checkCondition(('a < 2 || 'a > 3 || 'b > 5) && 'a < 2, 'a < 2) - checkCondition('a < 2 && ('a < 2 || 'a > 3 || 'b > 5) , 'a < 2) + checkCondition('a < 2 && ('a < 2 || 'a > 3 || 'b > 5), 'a < 2) checkCondition(('a < 2 || 'b > 3) && ('a < 2 || 'c > 5), 'a < 2 || ('b > 3 && 'c > 5)) 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 2dd82358fbfdf..b909765a7c6dd 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 @@ -945,7 +945,7 @@ class SQLContext private[sql]( } } - // Register a succesfully instantiatd context to the singleton. This should be at the end of + // Register a successfully instantiated context to the singleton. This should be at the end of // the class definition so that the singleton is updated only if there is no exception in the // construction of the instance. sparkContext.addSparkListener(new SparkListener { 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 6b100577077c6..058d147c7d65d 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 @@ -223,7 +223,7 @@ case class Exchange( new ShuffledRowRDD(shuffleDependency, specifiedPartitionStartIndices) } - protected override def doExecute(): RDD[InternalRow] = attachTree(this , "execute") { + protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") { coordinator match { case Some(exchangeCoordinator) => val shuffleRDD = exchangeCoordinator.postShuffleRDD(this) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala index 3a283a4e1f610..848f1af65508b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala @@ -27,7 +27,7 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("persist and unpersist") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS().select(expr("_2 + 1").as[Int]) + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS().select(expr("_2 + 1").as[Int]) val cached = ds.cache() // count triggers the caching action. It should not throw. cached.count() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 53b5f45c2d4a6..693f5aea2d015 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -30,7 +30,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("toDS") { - val data = Seq(("a", 1) , ("b", 2), ("c", 3)) + val data = Seq(("a", 1), ("b", 2), ("c", 3)) checkAnswer( data.toDS(), data: _*) @@ -87,7 +87,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("as case class / collect") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDF("a", "b").as[ClassData] + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDF("a", "b").as[ClassData] checkAnswer( ds, ClassData("a", 1), ClassData("b", 2), ClassData("c", 3)) @@ -105,7 +105,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("map") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.map(v => (v._1, v._2 + 1)), ("a", 2), ("b", 3), ("c", 4)) @@ -124,14 +124,14 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select(expr("_2 + 1").as[Int]), 2, 3, 4) } test("select 2") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select( expr("_1").as[String], @@ -140,7 +140,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select 2, primitive and tuple") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select( expr("_1").as[String], @@ -149,7 +149,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select 2, primitive and class") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select( expr("_1").as[String], @@ -158,7 +158,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select 2, primitive and class, fields reordered") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkDecoding( ds.select( expr("_1").as[String], @@ -167,28 +167,28 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("filter") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.filter(_._1 == "b"), ("b", 2)) } test("foreach") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() val acc = sparkContext.accumulator(0) ds.foreach(v => acc += v._2) assert(acc.value == 6) } test("foreachPartition") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() val acc = sparkContext.accumulator(0) ds.foreachPartition(_.foreach(v => acc += v._2)) assert(acc.value == 6) } test("reduce") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() assert(ds.reduce((a, b) => ("sum", a._2 + b._2)) == ("sum", 6)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 4ab148065a476..860e07c68cef1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -206,7 +206,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { StructType( StructField("f1", IntegerType, true) :: StructField("f2", IntegerType, true) :: Nil), - StructType(StructField("f1", LongType, true) :: Nil) , + StructType(StructField("f1", LongType, true) :: Nil), StructType( StructField("f1", LongType, true) :: StructField("f2", IntegerType, true) :: Nil)) 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 03bc830df2034..f279b78f47c7d 100644 --- 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 @@ -369,7 +369,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (counter != 0) { responseMsg += s", Fetched $counter row(s)" } - console.printInfo(responseMsg , null) + console.printInfo(responseMsg, null) // Destroy the driver to release all the locks. driver.destroy() } else { 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 da7303c791064..40e9c9362cf5e 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 @@ -154,8 +154,8 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef } 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=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)) From 3d81d63f4499478ef7861bf77383c30aed14bb19 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 13 Jan 2016 00:51:24 -0800 Subject: [PATCH 091/110] [SPARK-12692][BUILD] Enforce style checking about white space before comma This is the final PR about SPARK-12692. We have removed all of white spaces before comma from code so let's enforce style checking. Author: Kousuke Saruta Closes #10736 from sarutak/SPARK-12692-followup-enforce-checking. --- scalastyle-config.xml | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/scalastyle-config.xml b/scalastyle-config.xml index bc209ee6aa873..967a482ba4f9b 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -197,6 +197,12 @@ This file is divided into 3 sections: + + + COMMA + + + @@ -217,13 +223,6 @@ This file is divided into 3 sections: - - - - COMMA - - - From d6fd9b376b7071aecef34dc82a33eba42b183bc9 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 13 Jan 2016 10:01:15 -0800 Subject: [PATCH 092/110] [SPARK-12692][BUILD][HOT-FIX] Fix the scala style of KinesisBackedBlockRDDSuite.scala. https://github.com/apache/spark/pull/10736 was merged yesterday and caused the master start to fail because of the style issue. Author: Yin Huai Closes #10742 from yhuai/fixStyle. --- .../spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala index e6f504c4e54dd..e916f1ee0893b 100644 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala +++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala @@ -158,9 +158,9 @@ abstract class KinesisBackedBlockRDDTests(aggregateTestData: Boolean) testBlockRemove: Boolean = false ): Unit = { require(shardIds.size > 1, "Need at least 2 shards to test") - require(numPartitionsInBM <= shardIds.size , + require(numPartitionsInBM <= shardIds.size, "Number of partitions in BlockManager cannot be more than the Kinesis test shards available") - require(numPartitionsInKinesis <= shardIds.size , + require(numPartitionsInKinesis <= shardIds.size, "Number of partitions in Kinesis cannot be more than the Kinesis test shards available") require(numPartitionsInBM <= numPartitions, "Number of partitions in BlockManager cannot be more than that in RDD") From 63eee86cc652c108ca7712c8c0a73db1ca89ae90 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 13 Jan 2016 10:26:55 -0800 Subject: [PATCH 093/110] [SPARK-9297] [SQL] Add covar_pop and covar_samp JIRA: https://issues.apache.org/jira/browse/SPARK-9297 Add two aggregation functions: covar_pop and covar_samp. Author: Liang-Chi Hsieh Author: Liang-Chi Hsieh Closes #10029 from viirya/covar-funcs. --- .../catalyst/analysis/FunctionRegistry.scala | 2 + .../expressions/aggregate/Covariance.scala | 198 ++++++++++++++++++ .../org/apache/spark/sql/functions.scala | 40 ++++ .../execution/AggregationQuerySuite.scala | 32 +++ 4 files changed, 272 insertions(+) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala 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 5c2aa3c06b3e7..d9009e3848e58 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 @@ -182,6 +182,8 @@ object FunctionRegistry { expression[Average]("avg"), expression[Corr]("corr"), expression[Count]("count"), + expression[CovPopulation]("covar_pop"), + expression[CovSample]("covar_samp"), expression[First]("first"), expression[First]("first_value"), expression[Last]("last"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala new file mode 100644 index 0000000000000..f53b01be2a0d5 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.aggregate + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.util.TypeUtils +import org.apache.spark.sql.types._ + +/** + * Compute the covariance between two expressions. + * When applied on empty data (i.e., count is zero), it returns NULL. + * + */ +abstract class Covariance(left: Expression, right: Expression) extends ImperativeAggregate + with Serializable { + override def children: Seq[Expression] = Seq(left, right) + + override def nullable: Boolean = true + + override def dataType: DataType = DoubleType + + override def inputTypes: Seq[AbstractDataType] = Seq(DoubleType, DoubleType) + + override def checkInputDataTypes(): TypeCheckResult = { + if (left.dataType.isInstanceOf[DoubleType] && right.dataType.isInstanceOf[DoubleType]) { + TypeCheckResult.TypeCheckSuccess + } else { + TypeCheckResult.TypeCheckFailure( + s"covariance requires that both arguments are double type, " + + s"not (${left.dataType}, ${right.dataType}).") + } + } + + override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes) + + override def inputAggBufferAttributes: Seq[AttributeReference] = { + aggBufferAttributes.map(_.newInstance()) + } + + override val aggBufferAttributes: Seq[AttributeReference] = Seq( + AttributeReference("xAvg", DoubleType)(), + AttributeReference("yAvg", DoubleType)(), + AttributeReference("Ck", DoubleType)(), + AttributeReference("count", LongType)()) + + // Local cache of mutableAggBufferOffset(s) that will be used in update and merge + val xAvgOffset = mutableAggBufferOffset + val yAvgOffset = mutableAggBufferOffset + 1 + val CkOffset = mutableAggBufferOffset + 2 + val countOffset = mutableAggBufferOffset + 3 + + // Local cache of inputAggBufferOffset(s) that will be used in update and merge + val inputXAvgOffset = inputAggBufferOffset + val inputYAvgOffset = inputAggBufferOffset + 1 + val inputCkOffset = inputAggBufferOffset + 2 + val inputCountOffset = inputAggBufferOffset + 3 + + override def initialize(buffer: MutableRow): Unit = { + buffer.setDouble(xAvgOffset, 0.0) + buffer.setDouble(yAvgOffset, 0.0) + buffer.setDouble(CkOffset, 0.0) + buffer.setLong(countOffset, 0L) + } + + override def update(buffer: MutableRow, input: InternalRow): Unit = { + val leftEval = left.eval(input) + val rightEval = right.eval(input) + + if (leftEval != null && rightEval != null) { + val x = leftEval.asInstanceOf[Double] + val y = rightEval.asInstanceOf[Double] + + var xAvg = buffer.getDouble(xAvgOffset) + var yAvg = buffer.getDouble(yAvgOffset) + var Ck = buffer.getDouble(CkOffset) + var count = buffer.getLong(countOffset) + + val deltaX = x - xAvg + val deltaY = y - yAvg + count += 1 + xAvg += deltaX / count + yAvg += deltaY / count + Ck += deltaX * (y - yAvg) + + buffer.setDouble(xAvgOffset, xAvg) + buffer.setDouble(yAvgOffset, yAvg) + buffer.setDouble(CkOffset, Ck) + buffer.setLong(countOffset, count) + } + } + + // Merge counters from other partitions. Formula can be found at: + // http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance + override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = { + val count2 = buffer2.getLong(inputCountOffset) + + // We only go to merge two buffers if there is at least one record aggregated in buffer2. + // We don't need to check count in buffer1 because if count2 is more than zero, totalCount + // is more than zero too, then we won't get a divide by zero exception. + if (count2 > 0) { + var xAvg = buffer1.getDouble(xAvgOffset) + var yAvg = buffer1.getDouble(yAvgOffset) + var Ck = buffer1.getDouble(CkOffset) + var count = buffer1.getLong(countOffset) + + val xAvg2 = buffer2.getDouble(inputXAvgOffset) + val yAvg2 = buffer2.getDouble(inputYAvgOffset) + val Ck2 = buffer2.getDouble(inputCkOffset) + + val totalCount = count + count2 + val deltaX = xAvg - xAvg2 + val deltaY = yAvg - yAvg2 + Ck += Ck2 + deltaX * deltaY * count / totalCount * count2 + xAvg = (xAvg * count + xAvg2 * count2) / totalCount + yAvg = (yAvg * count + yAvg2 * count2) / totalCount + count = totalCount + + buffer1.setDouble(xAvgOffset, xAvg) + buffer1.setDouble(yAvgOffset, yAvg) + buffer1.setDouble(CkOffset, Ck) + buffer1.setLong(countOffset, count) + } + } +} + +case class CovSample( + left: Expression, + right: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) + extends Covariance(left, right) { + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = + copy(inputAggBufferOffset = newInputAggBufferOffset) + + override def eval(buffer: InternalRow): Any = { + val count = buffer.getLong(countOffset) + if (count > 1) { + val Ck = buffer.getDouble(CkOffset) + val cov = Ck / (count - 1) + if (cov.isNaN) { + null + } else { + cov + } + } else { + null + } + } +} + +case class CovPopulation( + left: Expression, + right: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) + extends Covariance(left, right) { + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = + copy(inputAggBufferOffset = newInputAggBufferOffset) + + override def eval(buffer: InternalRow): Any = { + val count = buffer.getLong(countOffset) + if (count > 0) { + val Ck = buffer.getDouble(CkOffset) + if (Ck.isNaN) { + null + } else { + Ck / count + } + } else { + null + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 592d79df3109a..71fea2716bd9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -308,6 +308,46 @@ object functions extends LegacyFunctions { def countDistinct(columnName: String, columnNames: String*): Column = countDistinct(Column(columnName), columnNames.map(Column.apply) : _*) + /** + * Aggregate function: returns the population covariance for two columns. + * + * @group agg_funcs + * @since 2.0.0 + */ + def covar_pop(column1: Column, column2: Column): Column = withAggregateFunction { + CovPopulation(column1.expr, column2.expr) + } + + /** + * Aggregate function: returns the population covariance for two columns. + * + * @group agg_funcs + * @since 2.0.0 + */ + def covar_pop(columnName1: String, columnName2: String): Column = { + covar_pop(Column(columnName1), Column(columnName2)) + } + + /** + * Aggregate function: returns the sample covariance for two columns. + * + * @group agg_funcs + * @since 2.0.0 + */ + def covar_samp(column1: Column, column2: Column): Column = withAggregateFunction { + CovSample(column1.expr, column2.expr) + } + + /** + * Aggregate function: returns the sample covariance for two columns. + * + * @group agg_funcs + * @since 2.0.0 + */ + def covar_samp(columnName1: String, columnName2: String): Column = { + covar_samp(Column(columnName1), Column(columnName2)) + } + /** * Aggregate function: returns the first value in a group. * diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index 5550198c02fbf..76b36aa89182e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -807,6 +807,38 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te assert(math.abs(corr7 - 0.6633880657639323) < 1e-12) } + test("covariance: covar_pop and covar_samp") { + // non-trivial example. To reproduce in python, use: + // >>> import numpy as np + // >>> a = np.array(range(20)) + // >>> b = np.array([x * x - 2 * x + 3.5 for x in range(20)]) + // >>> np.cov(a, b, bias = 0)[0][1] + // 595.0 + // >>> np.cov(a, b, bias = 1)[0][1] + // 565.25 + val df = Seq.tabulate(20)(x => (1.0 * x, x * x - 2 * x + 3.5)).toDF("a", "b") + val cov_samp = df.groupBy().agg(covar_samp("a", "b")).collect()(0).getDouble(0) + assert(math.abs(cov_samp - 595.0) < 1e-12) + + val cov_pop = df.groupBy().agg(covar_pop("a", "b")).collect()(0).getDouble(0) + assert(math.abs(cov_pop - 565.25) < 1e-12) + + val df2 = Seq.tabulate(20)(x => (1 * x, x * x * x - 2)).toDF("a", "b") + val cov_samp2 = df2.groupBy().agg(covar_samp("a", "b")).collect()(0).getDouble(0) + assert(math.abs(cov_samp2 - 11564.0) < 1e-12) + + val cov_pop2 = df2.groupBy().agg(covar_pop("a", "b")).collect()(0).getDouble(0) + assert(math.abs(cov_pop2 - 10985.799999999999) < 1e-12) + + // one row test + val df3 = Seq.tabulate(1)(x => (1 * x, x * x * x - 2)).toDF("a", "b") + val cov_samp3 = df3.groupBy().agg(covar_samp("a", "b")).collect()(0).get(0) + assert(cov_samp3 == null) + + val cov_pop3 = df3.groupBy().agg(covar_pop("a", "b")).collect()(0).getDouble(0) + assert(cov_pop3 == 0.0) + } + test("no aggregation function (SPARK-11486)") { val df = sqlContext.range(20).selectExpr("id", "repeat(id, 1) as s") .groupBy("s").count() From cc91e21879e031bcd05316eabb856e67a51b191d Mon Sep 17 00:00:00 2001 From: Luc Bourlier Date: Wed, 13 Jan 2016 11:45:13 -0800 Subject: [PATCH 094/110] [SPARK-12805][MESOS] Fixes documentation on Mesos run modes The default run has changed, but the documentation didn't fully reflect the change. Author: Luc Bourlier Closes #10740 from skyluc/issue/mesos-modes-doc. --- docs/running-on-mesos.md | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 3193e17853483..ed720f1039f94 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -202,7 +202,7 @@ where each application gets more or fewer machines as it ramps up and down, but additional overhead in launching each task. This mode may be inappropriate for low-latency requirements like interactive queries or serving web requests. -To run in coarse-grained mode, set the `spark.mesos.coarse` property to false in your +To run in fine-grained mode, set the `spark.mesos.coarse` property to false in your [SparkConf](configuration.html#spark-properties): {% highlight scala %} @@ -266,13 +266,11 @@ See the [configuration page](configuration.html) for information on Spark config

    - + From 38148f7373ee678cd538ce5eae0a75e15c62db8a Mon Sep 17 00:00:00 2001 From: Jakob Odersky Date: Wed, 13 Jan 2016 11:53:59 -0800 Subject: [PATCH 095/110] [SPARK-12761][CORE] Remove duplicated code Removes some duplicated code that was reintroduced during a merge. Author: Jakob Odersky Closes #10711 from jodersky/repl-2.11-duplicate. --- .../src/main/scala/org/apache/spark/repl/Main.scala | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) 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 index 44650f25f7a18..bb3081d12938e 100644 --- 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 @@ -30,11 +30,7 @@ object Main extends Logging { val conf = new SparkConf() val rootDir = conf.getOption("spark.repl.classdir").getOrElse(Utils.getLocalDir(conf)) val outputDir = Utils.createTempDir(root = rootDir, namePrefix = "repl") - val s = new Settings() - s.processArguments(List("-Yrepl-class-based", - "-Yrepl-outdir", s"${outputDir.getAbsolutePath}", - "-classpath", getAddedJars.mkString(File.pathSeparator)), true) - // the creation of SecurityManager has to be lazy so SPARK_YARN_MODE is set if needed + var sparkContext: SparkContext = _ var sqlContext: SQLContext = _ var interp = new SparkILoop // this is a public var because tests reset it. From 97e0c7c5af4d002937f9ee679568bb501d8818fc Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 13 Jan 2016 11:56:30 -0800 Subject: [PATCH 096/110] [SPARK-9383][PROJECT-INFRA] PR merge script should reset back to previous branch when possible This patch modifies our PR merge script to reset back to a named branch when restoring the original checkout upon exit. When the committer is originally checked out to a detached head, then they will be restored back to that same ref (the same as today's behavior). This is a slightly updated version of #7569, with an extra fix to handle the detached head corner-case. Author: Josh Rosen Closes #10709 from JoshRosen/SPARK-9383. --- dev/merge_spark_pr.py | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index bf1a000f46791..5ab285eae99b7 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -355,11 +355,21 @@ def standardize_jira_ref(text): return clean_text + +def get_current_ref(): + ref = run_cmd("git rev-parse --abbrev-ref HEAD").strip() + if ref == 'HEAD': + # The current ref is a detached HEAD, so grab its SHA. + return run_cmd("git rev-parse HEAD").strip() + else: + return ref + + def main(): global original_head os.chdir(SPARK_HOME) - original_head = run_cmd("git rev-parse HEAD")[:8] + original_head = get_current_ref() branches = get_json("%s/branches" % GITHUB_API_BASE) branch_names = filter(lambda x: x.startswith("branch-"), [x['name'] for x in branches]) @@ -449,5 +459,8 @@ def main(): (failure_count, test_count) = doctest.testmod() if failure_count: exit(-1) - - main() + try: + main() + except: + clean_up() + raise From e4e0b3f7b2945aae5ec7c3d68296010bbc5160cf Mon Sep 17 00:00:00 2001 From: Erik Selin Date: Wed, 13 Jan 2016 12:21:45 -0800 Subject: [PATCH 097/110] [SPARK-12268][PYSPARK] Make pyspark shell pythonstartup work under python3 This replaces the `execfile` used for running custom python shell scripts with explicit open, compile and exec (as recommended by 2to3). The reason for this change is to make the pythonstartup option compatible with python3. Author: Erik Selin Closes #10255 from tyro89/pythonstartup-python3. --- python/pyspark/shell.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 99331297c19f0..26cafca8b8381 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -76,4 +76,6 @@ # which allows us to execute the user's PYTHONSTARTUP file: _pythonstartup = os.environ.get('OLD_PYTHONSTARTUP') if _pythonstartup and os.path.isfile(_pythonstartup): - execfile(_pythonstartup) + with open(_pythonstartup) as f: + code = compile(f.read(), _pythonstartup, 'exec') + exec(code) From c2ea79f96acd076351b48162644ed1cff4c8e090 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 13 Jan 2016 12:29:02 -0800 Subject: [PATCH 098/110] [SPARK-12642][SQL] improve the hash expression to be decoupled from unsafe row https://issues.apache.org/jira/browse/SPARK-12642 Author: Wenchen Fan Closes #10694 from cloud-fan/hash-expr. --- python/pyspark/sql/functions.py | 2 +- .../sql/catalyst/expressions/UnsafeRow.java | 4 - .../spark/sql/catalyst/expressions/misc.scala | 251 +++++++++++++++++- .../expressions/MiscFunctionsSuite.scala | 6 +- .../sql/sources/BucketedWriteSuite.scala | 26 +- .../spark/unsafe/hash/Murmur3_x86_32.java | 28 +- 6 files changed, 288 insertions(+), 29 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index b0390cb9942e6..719eca8f5559e 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1023,7 +1023,7 @@ def hash(*cols): """Calculates the hash code of given columns, and returns the result as a int column. >>> sqlContext.createDataFrame([('ABC',)], ['a']).select(hash('a').alias('hash')).collect() - [Row(hash=1358996357)] + [Row(hash=-757602832)] """ sc = SparkContext._active_spark_context jc = sc._jvm.functions.hash(_to_seq(sc, cols, _to_java_column)) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index b8d3c49100476..1a351933a366c 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -566,10 +566,6 @@ public int hashCode() { return Murmur3_x86_32.hashUnsafeWords(baseObject, baseOffset, sizeInBytes, 42); } - public int hashCode(int seed) { - return Murmur3_x86_32.hashUnsafeWords(baseObject, baseOffset, sizeInBytes, seed); - } - @Override public boolean equals(Object other) { if (other instanceof UnsafeRow) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index cc406a39f0408..4751fbe4146fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -25,8 +25,11 @@ import org.apache.commons.codec.digest.DigestUtils import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.hash.Murmur3_x86_32 +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.Platform /** * A function that calculates an MD5 128-bit checksum and returns it as a hex string @@ -184,8 +187,31 @@ case class Crc32(child: Expression) extends UnaryExpression with ImplicitCastInp * A function that calculates hash value for a group of expressions. Note that the `seed` argument * is not exposed to users and should only be set inside spark SQL. * - * Internally this function will write arguments into an [[UnsafeRow]], and calculate hash code of - * the unsafe row using murmur3 hasher with a seed. + * The hash value for an expression depends on its type and seed: + * - null: seed + * - boolean: turn boolean into int, 1 for true, 0 for false, and then use murmur3 to + * hash this int with seed. + * - byte, short, int: use murmur3 to hash the input as int with seed. + * - long: use murmur3 to hash the long input with seed. + * - float: turn it into int: java.lang.Float.floatToIntBits(input), and hash it. + * - double: turn it into long: java.lang.Double.doubleToLongBits(input), and hash it. + * - decimal: if it's a small decimal, i.e. precision <= 18, turn it into long and hash + * it. Else, turn it into bytes and hash it. + * - calendar interval: hash `microseconds` first, and use the result as seed to hash `months`. + * - binary: use murmur3 to hash the bytes with seed. + * - string: get the bytes of string and hash it. + * - array: The `result` starts with seed, then use `result` as seed, recursively + * calculate hash value for each element, and assign the element hash value + * to `result`. + * - map: The `result` starts with seed, then use `result` as seed, recursively + * calculate hash value for each key-value, and assign the key-value hash + * value to `result`. + * - struct: The `result` starts with seed, then use `result` as seed, recursively + * calculate hash value for each field, and assign the field hash value to + * `result`. + * + * Finally we aggregate the hash values for each expression by the same way of struct. + * * We should use this hash function for both shuffle and bucket, so that we can guarantee shuffle * and bucketing have same data distribution. */ @@ -206,22 +232,225 @@ case class Murmur3Hash(children: Seq[Expression], seed: Int) extends Expression } } - private lazy val unsafeProjection = UnsafeProjection.create(children) + override def prettyName: String = "hash" + + override def sql: String = s"$prettyName(${children.map(_.sql).mkString(", ")}, $seed)" override def eval(input: InternalRow): Any = { - unsafeProjection(input).hashCode(seed) + var hash = seed + var i = 0 + val len = children.length + while (i < len) { + hash = computeHash(children(i).eval(input), children(i).dataType, hash) + i += 1 + } + hash } + private def computeHash(value: Any, dataType: DataType, seed: Int): Int = { + def hashInt(i: Int): Int = Murmur3_x86_32.hashInt(i, seed) + def hashLong(l: Long): Int = Murmur3_x86_32.hashLong(l, seed) + + value match { + case null => seed + case b: Boolean => hashInt(if (b) 1 else 0) + case b: Byte => hashInt(b) + case s: Short => hashInt(s) + case i: Int => hashInt(i) + case l: Long => hashLong(l) + case f: Float => hashInt(java.lang.Float.floatToIntBits(f)) + case d: Double => hashLong(java.lang.Double.doubleToLongBits(d)) + case d: Decimal => + val precision = dataType.asInstanceOf[DecimalType].precision + if (precision <= Decimal.MAX_LONG_DIGITS) { + hashLong(d.toUnscaledLong) + } else { + val bytes = d.toJavaBigDecimal.unscaledValue().toByteArray + Murmur3_x86_32.hashUnsafeBytes(bytes, Platform.BYTE_ARRAY_OFFSET, bytes.length, seed) + } + case c: CalendarInterval => Murmur3_x86_32.hashInt(c.months, hashLong(c.microseconds)) + case a: Array[Byte] => + Murmur3_x86_32.hashUnsafeBytes(a, Platform.BYTE_ARRAY_OFFSET, a.length, seed) + case s: UTF8String => + Murmur3_x86_32.hashUnsafeBytes(s.getBaseObject, s.getBaseOffset, s.numBytes(), seed) + + case array: ArrayData => + val elementType = dataType match { + case udt: UserDefinedType[_] => udt.sqlType.asInstanceOf[ArrayType].elementType + case ArrayType(et, _) => et + } + var result = seed + var i = 0 + while (i < array.numElements()) { + result = computeHash(array.get(i, elementType), elementType, result) + i += 1 + } + result + + case map: MapData => + val (kt, vt) = dataType match { + case udt: UserDefinedType[_] => + val mapType = udt.sqlType.asInstanceOf[MapType] + mapType.keyType -> mapType.valueType + case MapType(kt, vt, _) => kt -> vt + } + val keys = map.keyArray() + val values = map.valueArray() + var result = seed + var i = 0 + while (i < map.numElements()) { + result = computeHash(keys.get(i, kt), kt, result) + result = computeHash(values.get(i, vt), vt, result) + i += 1 + } + result + + case struct: InternalRow => + val types: Array[DataType] = dataType match { + case udt: UserDefinedType[_] => + udt.sqlType.asInstanceOf[StructType].map(_.dataType).toArray + case StructType(fields) => fields.map(_.dataType) + } + var result = seed + var i = 0 + val len = struct.numFields + while (i < len) { + result = computeHash(struct.get(i, types(i)), types(i), result) + i += 1 + } + result + } + } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val unsafeRow = GenerateUnsafeProjection.createCode(ctx, children) ev.isNull = "false" + val childrenHash = children.zipWithIndex.map { + case (child, dt) => + val childGen = child.gen(ctx) + val childHash = computeHash(childGen.value, child.dataType, ev.value, ctx) + s""" + ${childGen.code} + if (!${childGen.isNull}) { + ${childHash.code} + ${ev.value} = ${childHash.value}; + } + """ + }.mkString("\n") s""" - ${unsafeRow.code} - final int ${ev.value} = ${unsafeRow.value}.hashCode($seed); + int ${ev.value} = $seed; + $childrenHash """ } - override def prettyName: String = "hash" - - override def sql: String = s"$prettyName(${children.map(_.sql).mkString(", ")}, $seed)" + private def computeHash( + input: String, + dataType: DataType, + seed: String, + ctx: CodeGenContext): GeneratedExpressionCode = { + val hasher = classOf[Murmur3_x86_32].getName + def hashInt(i: String): GeneratedExpressionCode = inlineValue(s"$hasher.hashInt($i, $seed)") + def hashLong(l: String): GeneratedExpressionCode = inlineValue(s"$hasher.hashLong($l, $seed)") + def inlineValue(v: String): GeneratedExpressionCode = + GeneratedExpressionCode(code = "", isNull = "false", value = v) + + dataType match { + case NullType => inlineValue(seed) + case BooleanType => hashInt(s"$input ? 1 : 0") + case ByteType | ShortType | IntegerType | DateType => hashInt(input) + case LongType | TimestampType => hashLong(input) + case FloatType => hashInt(s"Float.floatToIntBits($input)") + case DoubleType => hashLong(s"Double.doubleToLongBits($input)") + case d: DecimalType => + if (d.precision <= Decimal.MAX_LONG_DIGITS) { + hashLong(s"$input.toUnscaledLong()") + } else { + val bytes = ctx.freshName("bytes") + val code = s"byte[] $bytes = $input.toJavaBigDecimal().unscaledValue().toByteArray();" + val offset = "Platform.BYTE_ARRAY_OFFSET" + val result = s"$hasher.hashUnsafeBytes($bytes, $offset, $bytes.length, $seed)" + GeneratedExpressionCode(code, "false", result) + } + case CalendarIntervalType => + val microsecondsHash = s"$hasher.hashLong($input.microseconds, $seed)" + val monthsHash = s"$hasher.hashInt($input.months, $microsecondsHash)" + inlineValue(monthsHash) + case BinaryType => + val offset = "Platform.BYTE_ARRAY_OFFSET" + inlineValue(s"$hasher.hashUnsafeBytes($input, $offset, $input.length, $seed)") + case StringType => + val baseObject = s"$input.getBaseObject()" + val baseOffset = s"$input.getBaseOffset()" + val numBytes = s"$input.numBytes()" + inlineValue(s"$hasher.hashUnsafeBytes($baseObject, $baseOffset, $numBytes, $seed)") + + case ArrayType(et, _) => + val result = ctx.freshName("result") + val index = ctx.freshName("index") + val element = ctx.freshName("element") + val elementHash = computeHash(element, et, result, ctx) + val code = + s""" + int $result = $seed; + for (int $index = 0; $index < $input.numElements(); $index++) { + if (!$input.isNullAt($index)) { + final ${ctx.javaType(et)} $element = ${ctx.getValue(input, et, index)}; + ${elementHash.code} + $result = ${elementHash.value}; + } + } + """ + GeneratedExpressionCode(code, "false", result) + + case MapType(kt, vt, _) => + val result = ctx.freshName("result") + val index = ctx.freshName("index") + val keys = ctx.freshName("keys") + val values = ctx.freshName("values") + val key = ctx.freshName("key") + val value = ctx.freshName("value") + val keyHash = computeHash(key, kt, result, ctx) + val valueHash = computeHash(value, vt, result, ctx) + val code = + s""" + int $result = $seed; + final ArrayData $keys = $input.keyArray(); + final ArrayData $values = $input.valueArray(); + for (int $index = 0; $index < $input.numElements(); $index++) { + final ${ctx.javaType(kt)} $key = ${ctx.getValue(keys, kt, index)}; + ${keyHash.code} + $result = ${keyHash.value}; + if (!$values.isNullAt($index)) { + final ${ctx.javaType(vt)} $value = ${ctx.getValue(values, vt, index)}; + ${valueHash.code} + $result = ${valueHash.value}; + } + } + """ + GeneratedExpressionCode(code, "false", result) + + case StructType(fields) => + val result = ctx.freshName("result") + val fieldsHash = fields.map(_.dataType).zipWithIndex.map { + case (dt, index) => + val field = ctx.freshName("field") + val fieldHash = computeHash(field, dt, result, ctx) + s""" + if (!$input.isNullAt($index)) { + final ${ctx.javaType(dt)} $field = ${ctx.getValue(input, dt, index.toString)}; + ${fieldHash.code} + $result = ${fieldHash.value}; + } + """ + }.mkString("\n") + val code = + s""" + int $result = $seed; + $fieldsHash + """ + GeneratedExpressionCode(code, "false", result) + + case udt: UserDefinedType[_] => computeHash(input, udt.sqlType, seed, ctx) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala index 64161bebdcbe8..75131a6170222 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala @@ -79,7 +79,8 @@ class MiscFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { .add("long", LongType) .add("float", FloatType) .add("double", DoubleType) - .add("decimal", DecimalType.SYSTEM_DEFAULT) + .add("bigDecimal", DecimalType.SYSTEM_DEFAULT) + .add("smallDecimal", DecimalType.USER_DEFAULT) .add("string", StringType) .add("binary", BinaryType) .add("date", DateType) @@ -126,7 +127,8 @@ class MiscFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { val literals = input.toSeq(inputSchema).zip(inputSchema.map(_.dataType)).map { case (value, dt) => Literal.create(value, dt) } - checkEvaluation(Murmur3Hash(literals, seed), input.hashCode(seed)) + // Only test the interpreted version has same result with codegen version. + checkEvaluation(Murmur3Hash(literals, seed), Murmur3Hash(literals, seed).eval()) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala index 7f1745705aaaf..b718b7cefb2a4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala @@ -20,10 +20,11 @@ package org.apache.spark.sql.sources import java.io.File import org.apache.spark.sql.{AnalysisException, QueryTest} -import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.expressions.{Murmur3Hash, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.util.Utils class BucketedWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { import testImplicits._ @@ -70,6 +71,8 @@ class BucketedWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingle } } + private val df = (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k") + private def testBucketing( dataDir: File, source: String, @@ -82,27 +85,30 @@ class BucketedWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingle assert(groupedBucketFiles.size <= 8) for ((bucketId, bucketFiles) <- groupedBucketFiles) { - for (bucketFile <- bucketFiles) { - val df = sqlContext.read.format(source).load(bucketFile.getAbsolutePath) - .select((bucketCols ++ sortCols).map(col): _*) + for (bucketFilePath <- bucketFiles.map(_.getAbsolutePath)) { + val types = df.select((bucketCols ++ sortCols).map(col): _*).schema.map(_.dataType) + val columns = (bucketCols ++ sortCols).zip(types).map { + case (colName, dt) => col(colName).cast(dt) + } + val readBack = sqlContext.read.format(source).load(bucketFilePath).select(columns: _*) if (sortCols.nonEmpty) { - checkAnswer(df.sort(sortCols.map(col): _*), df.collect()) + checkAnswer(readBack.sort(sortCols.map(col): _*), readBack.collect()) } - val rows = df.select(bucketCols.map(col): _*).queryExecution.toRdd.map(_.copy()).collect() + val qe = readBack.select(bucketCols.map(col): _*).queryExecution + val rows = qe.toRdd.map(_.copy()).collect() + val getHashCode = + UnsafeProjection.create(new Murmur3Hash(qe.analyzed.output) :: Nil, qe.analyzed.output) for (row <- rows) { - assert(row.isInstanceOf[UnsafeRow]) - val actualBucketId = (row.hashCode() % 8 + 8) % 8 + val actualBucketId = Utils.nonNegativeMod(getHashCode(row).getInt(0), 8) assert(actualBucketId == bucketId) } } } } - private val df = (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k") - test("write bucketed data") { for (source <- Seq("parquet", "json", "orc")) { withTable("bucketed_table") { diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index 4276f25c2165b..5e7ee480cafd1 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -38,6 +38,10 @@ public String toString() { } public int hashInt(int input) { + return hashInt(input, seed); + } + + public static int hashInt(int input, int seed) { int k1 = mixK1(input); int h1 = mixH1(seed, k1); @@ -51,16 +55,38 @@ public int hashUnsafeWords(Object base, long offset, int lengthInBytes) { public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, int seed) { // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method. assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)"; + int h1 = hashBytesByInt(base, offset, lengthInBytes, seed); + return fmix(h1, lengthInBytes); + } + + public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes, int seed) { + assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; + int lengthAligned = lengthInBytes - lengthInBytes % 4; + int h1 = hashBytesByInt(base, offset, lengthAligned, seed); + for (int i = lengthAligned; i < lengthInBytes; i++) { + int halfWord = Platform.getByte(base, offset + i); + int k1 = mixK1(halfWord); + h1 = mixH1(h1, k1); + } + return fmix(h1, lengthInBytes); + } + + private static int hashBytesByInt(Object base, long offset, int lengthInBytes, int seed) { + assert (lengthInBytes % 4 == 0); int h1 = seed; for (int i = 0; i < lengthInBytes; i += 4) { int halfWord = Platform.getInt(base, offset + i); int k1 = mixK1(halfWord); h1 = mixH1(h1, k1); } - return fmix(h1, lengthInBytes); + return h1; } public int hashLong(long input) { + return hashLong(input, seed); + } + + public static int hashLong(long input, int seed) { int low = (int) input; int high = (int) (input >>> 32); From cbbcd8e4250aeec700f04c231f8be2f787243f1f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 13 Jan 2016 12:44:35 -0800 Subject: [PATCH 099/110] [SPARK-12791][SQL] Simplify CaseWhen by breaking "branches" into "conditions" and "values" This pull request rewrites CaseWhen expression to break the single, monolithic "branches" field into a sequence of tuples (Seq[(condition, value)]) and an explicit optional elseValue field. Prior to this pull request, each even position in "branches" represents the condition for each branch, and each odd position represents the value for each branch. The use of them have been pretty confusing with a lot sliding windows or grouped(2) calls. Author: Reynold Xin Closes #10734 from rxin/simplify-case. --- python/pyspark/sql/column.py | 24 +-- .../spark/sql/catalyst/CatalystQl.scala | 2 +- .../apache/spark/sql/catalyst/SqlParser.scala | 3 +- .../catalyst/analysis/HiveTypeCoercion.scala | 26 +++- .../expressions/conditionalExpressions.scala | 137 +++++++++--------- .../spark/sql/catalyst/trees/TreeNode.scala | 9 ++ .../sql/catalyst/analysis/AnalysisSuite.scala | 2 +- .../ExpressionTypeCheckingSuite.scala | 4 +- .../analysis/HiveTypeCoercionSuite.scala | 15 +- .../ConditionalExpressionSuite.scala | 51 +++---- .../scala/org/apache/spark/sql/Column.scala | 19 +-- .../org/apache/spark/sql/functions.scala | 2 +- 12 files changed, 156 insertions(+), 138 deletions(-) diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index 900def59d23a5..320451c52c706 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -368,12 +368,12 @@ def when(self, condition, value): >>> from pyspark.sql import functions as F >>> df.select(df.name, F.when(df.age > 4, 1).when(df.age < 3, -1).otherwise(0)).show() - +-----+--------------------------------------------------------+ - | name|CASE WHEN (age > 4) THEN 1 WHEN (age < 3) THEN -1 ELSE 0| - +-----+--------------------------------------------------------+ - |Alice| -1| - | Bob| 1| - +-----+--------------------------------------------------------+ + +-----+------------------------------------------------------------+ + | name|CASE WHEN (age > 4) THEN 1 WHEN (age < 3) THEN -1 ELSE 0 END| + +-----+------------------------------------------------------------+ + |Alice| -1| + | Bob| 1| + +-----+------------------------------------------------------------+ """ if not isinstance(condition, Column): raise TypeError("condition should be a Column") @@ -393,12 +393,12 @@ def otherwise(self, value): >>> from pyspark.sql import functions as F >>> df.select(df.name, F.when(df.age > 3, 1).otherwise(0)).show() - +-----+---------------------------------+ - | name|CASE WHEN (age > 3) THEN 1 ELSE 0| - +-----+---------------------------------+ - |Alice| 0| - | Bob| 1| - +-----+---------------------------------+ + +-----+-------------------------------------+ + | name|CASE WHEN (age > 3) THEN 1 ELSE 0 END| + +-----+-------------------------------------+ + |Alice| 0| + | Bob| 1| + +-----+-------------------------------------+ """ v = value._jc if isinstance(value, Column) else value jc = self._jc.otherwise(v) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala index c87b6c8e95436..d0fbdacf6eafd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala @@ -752,7 +752,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C /* Case statements */ case Token("TOK_FUNCTION", Token(WHEN(), Nil) :: branches) => - CaseWhen(branches.map(nodeToExpr)) + CaseWhen.createFromParser(branches.map(nodeToExpr)) case Token("TOK_FUNCTION", Token(CASE(), Nil) :: branches) => val keyExpr = nodeToExpr(branches.head) CaseKeyWhen(keyExpr, branches.drop(1).map(nodeToExpr)) 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 6ec408a673c79..85ff4ea0c946b 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 @@ -305,7 +305,8 @@ object SqlParser extends AbstractSparkSQLParser with DataTypeParser { throw new AnalysisException(s"invalid function approximate($s) $udfName") } } - | CASE ~> whenThenElse ^^ CaseWhen + | CASE ~> whenThenElse ^^ + { case branches => CaseWhen.createFromParser(branches) } | CASE ~> expression ~ whenThenElse ^^ { case keyPart ~ branches => CaseKeyWhen(keyPart, branches) } ) 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 980b5d52fa8f7..2737fe32cd086 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 @@ -621,14 +621,24 @@ object HiveTypeCoercion { case c: CaseWhen if c.childrenResolved && !c.valueTypesEqual => val maybeCommonType = findWiderCommonType(c.valueTypes) maybeCommonType.map { commonType => - val castedBranches = c.branches.grouped(2).map { - case Seq(when, value) if value.dataType != commonType => - Seq(when, Cast(value, commonType)) - case Seq(elseVal) if elseVal.dataType != commonType => - Seq(Cast(elseVal, commonType)) - case other => other - }.reduce(_ ++ _) - CaseWhen(castedBranches) + var changed = false + val newBranches = c.branches.map { case (condition, value) => + if (value.dataType.sameType(commonType)) { + (condition, value) + } else { + changed = true + (condition, Cast(value, commonType)) + } + } + val newElseValue = c.elseValue.map { value => + if (value.dataType.sameType(commonType)) { + value + } else { + changed = true + Cast(value, commonType) + } + } + if (changed) CaseWhen(newBranches, newElseValue) else c }.getOrElse(c) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 5a1462433d583..8cc7bc1da2fc3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -81,44 +81,39 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi /** * Case statements of the form "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END". * When a = true, returns b; when c = true, returns d; else returns e. + * + * @param branches seq of (branch condition, branch value) + * @param elseValue optional value for the else branch */ -case class CaseWhen(branches: Seq[Expression]) extends Expression { - - // Use private[this] Array to speed up evaluation. - @transient private[this] lazy val branchesArr = branches.toArray - - override def children: Seq[Expression] = branches - - @transient lazy val whenList = - branches.sliding(2, 2).collect { case Seq(whenExpr, _) => whenExpr }.toSeq - - @transient lazy val thenList = - branches.sliding(2, 2).collect { case Seq(_, thenExpr) => thenExpr }.toSeq +case class CaseWhen(branches: Seq[(Expression, Expression)], elseValue: Option[Expression] = None) + extends Expression { - val elseValue = if (branches.length % 2 == 0) None else Option(branches.last) + override def children: Seq[Expression] = branches.flatMap(b => b._1 :: b._2 :: Nil) ++ elseValue // both then and else expressions should be considered. - def valueTypes: Seq[DataType] = (thenList ++ elseValue).map(_.dataType) + def valueTypes: Seq[DataType] = branches.map(_._2.dataType) ++ elseValue.map(_.dataType) + def valueTypesEqual: Boolean = valueTypes.size <= 1 || valueTypes.sliding(2, 1).forall { case Seq(dt1, dt2) => dt1.sameType(dt2) } - override def dataType: DataType = thenList.head.dataType + override def dataType: DataType = branches.head._2.dataType override def nullable: Boolean = { - // If no value is nullable and no elseValue is provided, the whole statement defaults to null. - thenList.exists(_.nullable) || elseValue.map(_.nullable).getOrElse(true) + // Result is nullable if any of the branch is nullable, or if the else value is nullable + branches.exists(_._2.nullable) || elseValue.map(_.nullable).getOrElse(true) } override def checkInputDataTypes(): TypeCheckResult = { + // Make sure all branch conditions are boolean types. if (valueTypesEqual) { - if (whenList.forall(_.dataType == BooleanType)) { + if (branches.forall(_._1.dataType == BooleanType)) { TypeCheckResult.TypeCheckSuccess } else { - val index = whenList.indexWhere(_.dataType != BooleanType) + val index = branches.indexWhere(_._1.dataType != BooleanType) TypeCheckResult.TypeCheckFailure( s"WHEN expressions in CaseWhen should all be boolean type, " + - s"but the ${index + 1}th when expression's type is ${whenList(index)}") + s"but the ${index + 1}th when expression's type is ${branches(index)._1}") } } else { TypeCheckResult.TypeCheckFailure( @@ -127,31 +122,26 @@ case class CaseWhen(branches: Seq[Expression]) extends Expression { } override def eval(input: InternalRow): Any = { - // Written in imperative fashion for performance considerations - val len = branchesArr.length var i = 0 - // If all branches fail and an elseVal is not provided, the whole statement - // defaults to null, according to Hive's semantics. - while (i < len - 1) { - if (branchesArr(i).eval(input) == true) { - return branchesArr(i + 1).eval(input) + while (i < branches.size) { + if (java.lang.Boolean.TRUE.equals(branches(i)._1.eval(input))) { + return branches(i)._2.eval(input) } - i += 2 + i += 1 } - var res: Any = null - if (i == len - 1) { - res = branchesArr(i).eval(input) + if (elseValue.isDefined) { + return elseValue.get.eval(input) + } else { + return null } - return res } override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val len = branchesArr.length val got = ctx.freshName("got") - val cases = (0 until len/2).map { i => - val cond = branchesArr(i * 2).gen(ctx) - val res = branchesArr(i * 2 + 1).gen(ctx) + val cases = branches.map { case (condition, value) => + val cond = condition.gen(ctx) + val res = value.gen(ctx) s""" if (!$got) { ${cond.code} @@ -165,17 +155,19 @@ case class CaseWhen(branches: Seq[Expression]) extends Expression { """ }.mkString("\n") - val other = if (len % 2 == 1) { - val res = branchesArr(len - 1).gen(ctx) - s""" + val elseCase = { + if (elseValue.isDefined) { + val res = elseValue.get.gen(ctx) + s""" if (!$got) { ${res.code} ${ev.isNull} = ${res.isNull}; ${ev.value} = ${res.value}; } - """ - } else { - "" + """ + } else { + "" + } } s""" @@ -183,32 +175,42 @@ case class CaseWhen(branches: Seq[Expression]) extends Expression { boolean ${ev.isNull} = true; ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; $cases - $other + $elseCase """ } override def toString: String = { - "CASE" + branches.sliding(2, 2).map { - case Seq(cond, value) => s" WHEN $cond THEN $value" - case Seq(elseValue) => s" ELSE $elseValue" - }.mkString + val cases = branches.map { case (c, v) => s" WHEN $c THEN $v" }.mkString + val elseCase = elseValue.map(" ELSE " + _).getOrElse("") + "CASE" + cases + elseCase + " END" } override def sql: String = { - val branchesSQL = branches.map(_.sql) - val (cases, maybeElse) = if (branches.length % 2 == 0) { - (branchesSQL, None) - } else { - (branchesSQL.init, Some(branchesSQL.last)) - } + val cases = branches.map { case (c, v) => s" WHEN ${c.sql} THEN ${v.sql}" }.mkString + val elseCase = elseValue.map(" ELSE " + _.sql).getOrElse("") + "CASE" + cases + elseCase + " END" + } +} - val head = s"CASE " - val tail = maybeElse.map(e => s" ELSE $e").getOrElse("") + " END" - val body = cases.grouped(2).map { - case Seq(whenExpr, thenExpr) => s"WHEN $whenExpr THEN $thenExpr" - }.mkString(" ") +/** Factory methods for CaseWhen. */ +object CaseWhen { - head + body + tail + def apply(branches: Seq[(Expression, Expression)], elseValue: Expression): CaseWhen = { + CaseWhen(branches, Option(elseValue)) + } + + /** + * A factory method to faciliate the creation of this expression when used in parsers. + * @param branches Expressions at even position are the branch conditions, and expressions at odd + * position are branch values. + */ + def createFromParser(branches: Seq[Expression]): CaseWhen = { + val cases = branches.grouped(2).flatMap { + case cond :: value :: Nil => Some((cond, value)) + case value :: Nil => None + }.toArray.toSeq // force materialization to make the seq serializable + val elseValue = if (branches.size % 2 == 1) Some(branches.last) else None + CaseWhen(cases, elseValue) } } @@ -218,17 +220,12 @@ case class CaseWhen(branches: Seq[Expression]) extends Expression { */ object CaseKeyWhen { def apply(key: Expression, branches: Seq[Expression]): CaseWhen = { - val newBranches = branches.zipWithIndex.map { case (expr, i) => - if (i % 2 == 0 && i != branches.size - 1) { - // If this expression is at even position, then it is either a branch condition, or - // the very last value that is the "else value". The "i != branches.size - 1" makes - // sure we are not adding an EqualTo to the "else value". - EqualTo(key, expr) - } else { - expr - } - } - CaseWhen(newBranches) + val cases = branches.grouped(2).flatMap { + case cond :: value :: Nil => Some((EqualTo(key, cond), value)) + case value :: Nil => None + }.toArray.toSeq // force materialization to make the seq serializable + val elseValue = if (branches.size % 2 == 1) Some(branches.last) else None + CaseWhen(cases, elseValue) } } 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 d4be545a35ab2..d0b29aa01f640 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 @@ -315,6 +315,15 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } else { arg } + case tuple @ (arg1: TreeNode[_], arg2: TreeNode[_]) => + val newChild1 = nextOperation(arg1.asInstanceOf[BaseType], rule) + val newChild2 = nextOperation(arg2.asInstanceOf[BaseType], rule) + if (!(newChild1 fastEquals arg1) || !(newChild2 fastEquals arg2)) { + changed = true + (newChild1, newChild2) + } else { + tuple + } case other => other } case nonChild: AnyRef => nonChild 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 cf84855885a37..975cd87d090e4 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 @@ -239,7 +239,7 @@ class AnalysisSuite extends AnalysisTest { test("SPARK-12102: Ignore nullablity when comparing two sides of case") { val relation = LocalRelation('a.struct('x.int), 'b.struct('x.int.withNullability(false))) - val plan = relation.select(CaseWhen(Seq(Literal(true), 'a, 'b)).as("val")) + val plan = relation.select(CaseWhen(Seq((Literal(true), 'a.attr)), 'b).as("val")) assertAnalysisSuccess(plan) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index 0521ed848c793..59549e3998e7e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -132,13 +132,13 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { assertErrorForDifferingTypes(If('booleanField, 'intField, 'booleanField)) assertError( - CaseWhen(Seq('booleanField, 'intField, 'booleanField, 'mapField)), + CaseWhen(Seq(('booleanField.attr, 'intField.attr), ('booleanField.attr, 'mapField.attr))), "THEN and ELSE expressions should all be same type or coercible to a common type") assertError( CaseKeyWhen('intField, Seq('intField, 'stringField, 'intField, 'mapField)), "THEN and ELSE expressions should all be same type or coercible to a common type") assertError( - CaseWhen(Seq('booleanField, 'intField, 'intField, 'intField)), + CaseWhen(Seq(('booleanField.attr, 'intField.attr), ('intField.attr, 'intField.attr))), "WHEN expressions in CaseWhen should all be boolean type") } 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 40378c6727667..b1f6c0b802d8e 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 @@ -308,15 +308,14 @@ class HiveTypeCoercionSuite extends PlanTest { CaseKeyWhen(Literal(true), Seq(Literal(1), Literal("a"))) ) ruleTest(HiveTypeCoercion.CaseWhenCoercion, - CaseWhen(Seq(Literal(true), Literal(1.2), Literal.create(1, DecimalType(7, 2)))), - CaseWhen(Seq( - Literal(true), Literal(1.2), Cast(Literal.create(1, DecimalType(7, 2)), DoubleType))) + CaseWhen(Seq((Literal(true), Literal(1.2))), Literal.create(1, DecimalType(7, 2))), + CaseWhen(Seq((Literal(true), Literal(1.2))), + Cast(Literal.create(1, DecimalType(7, 2)), DoubleType)) ) ruleTest(HiveTypeCoercion.CaseWhenCoercion, - CaseWhen(Seq(Literal(true), Literal(100L), Literal.create(1, DecimalType(7, 2)))), - CaseWhen(Seq( - Literal(true), Cast(Literal(100L), DecimalType(22, 2)), - Cast(Literal.create(1, DecimalType(7, 2)), DecimalType(22, 2)))) + CaseWhen(Seq((Literal(true), Literal(100L))), Literal.create(1, DecimalType(7, 2))), + CaseWhen(Seq((Literal(true), Cast(Literal(100L), DecimalType(22, 2)))), + Cast(Literal.create(1, DecimalType(7, 2)), DecimalType(22, 2))) ) } @@ -452,7 +451,7 @@ class HiveTypeCoercionSuite extends PlanTest { val expectedTypes = Seq(DecimalType(10, 5), DecimalType(10, 5), DecimalType(15, 5), DecimalType(25, 5), DoubleType, DoubleType) - rightTypes.zip(expectedTypes).map { case (rType, expectedType) => + rightTypes.zip(expectedTypes).foreach { case (rType, expectedType) => val plan2 = LocalRelation( AttributeReference("r", rType)()) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala index 4029da5925580..3c581ecdaf068 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala @@ -80,38 +80,39 @@ class ConditionalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val c5 = 'a.string.at(4) val c6 = 'a.string.at(5) - checkEvaluation(CaseWhen(Seq(c1, c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(c2, c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(c3, c4, c6)), "a", row) - checkEvaluation(CaseWhen(Seq(Literal.create(null, BooleanType), c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(Literal.create(false, BooleanType), c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(Literal.create(true, BooleanType), c4, c6)), "a", row) - - checkEvaluation(CaseWhen(Seq(c3, c4, c2, c5, c6)), "a", row) - checkEvaluation(CaseWhen(Seq(c2, c4, c3, c5, c6)), "b", row) - checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5)), null, row) - - assert(CaseWhen(Seq(c2, c4, c6)).nullable === true) - assert(CaseWhen(Seq(c2, c4, c3, c5, c6)).nullable === true) - assert(CaseWhen(Seq(c2, c4, c3, c5)).nullable === true) + checkEvaluation(CaseWhen(Seq((c1, c4)), c6), "c", row) + checkEvaluation(CaseWhen(Seq((c2, c4)), c6), "c", row) + checkEvaluation(CaseWhen(Seq((c3, c4)), c6), "a", row) + checkEvaluation(CaseWhen(Seq((Literal.create(null, BooleanType), c4)), c6), "c", row) + checkEvaluation(CaseWhen(Seq((Literal.create(false, BooleanType), c4)), c6), "c", row) + checkEvaluation(CaseWhen(Seq((Literal.create(true, BooleanType), c4)), c6), "a", row) + + checkEvaluation(CaseWhen(Seq((c3, c4), (c2, c5)), c6), "a", row) + checkEvaluation(CaseWhen(Seq((c2, c4), (c3, c5)), c6), "b", row) + checkEvaluation(CaseWhen(Seq((c1, c4), (c2, c5)), c6), "c", row) + checkEvaluation(CaseWhen(Seq((c1, c4), (c2, c5))), null, row) + + assert(CaseWhen(Seq((c2, c4)), c6).nullable === true) + assert(CaseWhen(Seq((c2, c4), (c3, c5)), c6).nullable === true) + assert(CaseWhen(Seq((c2, c4), (c3, c5))).nullable === true) val c4_notNull = 'a.boolean.notNull.at(3) val c5_notNull = 'a.boolean.notNull.at(4) val c6_notNull = 'a.boolean.notNull.at(5) - assert(CaseWhen(Seq(c2, c4_notNull, c6_notNull)).nullable === false) - assert(CaseWhen(Seq(c2, c4, c6_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c6)).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull)), c6_notNull).nullable === false) + assert(CaseWhen(Seq((c2, c4)), c6_notNull).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull))).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull)), c6).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6_notNull)).nullable === false) - assert(CaseWhen(Seq(c2, c4, c3, c5_notNull, c6_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5, c6_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6)).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5_notNull)), c6_notNull).nullable === false) + assert(CaseWhen(Seq((c2, c4), (c3, c5_notNull)), c6_notNull).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5)), c6_notNull).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5_notNull)), c6).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4, c3, c5_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5)).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5_notNull))).nullable === true) + assert(CaseWhen(Seq((c2, c4), (c3, c5_notNull))).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5))).nullable === true) } test("case key when") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index e8c61d6e01dc3..6a020f9f2883e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -437,8 +437,11 @@ class Column(protected[sql] val expr: Expression) extends Logging { * @since 1.4.0 */ def when(condition: Column, value: Any): Column = this.expr match { - case CaseWhen(branches: Seq[Expression]) => - withExpr { CaseWhen(branches ++ Seq(lit(condition).expr, lit(value).expr)) } + case CaseWhen(branches, None) => + withExpr { CaseWhen(branches :+ (condition.expr, lit(value).expr)) } + case CaseWhen(branches, Some(_)) => + throw new IllegalArgumentException( + "when() cannot be applied once otherwise() is applied") case _ => throw new IllegalArgumentException( "when() can only be applied on a Column previously generated by when() function") @@ -466,13 +469,11 @@ class Column(protected[sql] val expr: Expression) extends Logging { * @since 1.4.0 */ def otherwise(value: Any): Column = this.expr match { - case CaseWhen(branches: Seq[Expression]) => - if (branches.size % 2 == 0) { - withExpr { CaseWhen(branches :+ lit(value).expr) } - } else { - throw new IllegalArgumentException( - "otherwise() can only be applied once on a Column previously generated by when()") - } + case CaseWhen(branches, None) => + withExpr { CaseWhen(branches, Option(lit(value).expr)) } + case CaseWhen(branches, Some(_)) => + throw new IllegalArgumentException( + "otherwise() can only be applied once on a Column previously generated by when()") case _ => throw new IllegalArgumentException( "otherwise() can only be applied on a Column previously generated by when()") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 71fea2716bd9f..b8ea2261e94e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1042,7 +1042,7 @@ object functions extends LegacyFunctions { * @since 1.4.0 */ def when(condition: Column, value: Any): Column = withExpr { - CaseWhen(Seq(condition.expr, lit(value).expr)) + CaseWhen(Seq((condition.expr, lit(value).expr))) } /** From eabc7b8ee7e809bab05361ed154f87bff467bd88 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Wed, 13 Jan 2016 13:28:39 -0800 Subject: [PATCH 100/110] [SPARK-12690][CORE] Fix NPE in UnsafeInMemorySorter.free() I hit the exception below. The `UnsafeKVExternalSorter` does pass `null` as the consumer when creating an `UnsafeInMemorySorter`. Normally the NPE doesn't occur because the `inMemSorter` is set to null later and the `free()` method is not called. It happens when there is another exception like OOM thrown before setting `inMemSorter` to null. Anyway, we can add the null check to avoid it. ``` ERROR spark.TaskContextImpl: Error in TaskCompletionListener java.lang.NullPointerException at org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.free(UnsafeInMemorySorter.java:110) at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.cleanupResources(UnsafeExternalSorter.java:288) at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter$1.onTaskCompletion(UnsafeExternalSorter.java:141) at org.apache.spark.TaskContextImpl$$anonfun$markTaskCompleted$1.apply(TaskContextImpl.scala:79) at org.apache.spark.TaskContextImpl$$anonfun$markTaskCompleted$1.apply(TaskContextImpl.scala:77) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47) at org.apache.spark.TaskContextImpl.markTaskCompleted(TaskContextImpl.scala:77) at org.apache.spark.scheduler.Task.run(Task.scala:91) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) at java.lang.Thread.run(Thread.java:722) ``` Author: Carson Wang Closes #10637 from carsonwang/FixNPE. --- .../util/collection/unsafe/sort/UnsafeInMemorySorter.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index f71b8d154cc24..d1b0bc5d11f46 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -116,8 +116,10 @@ public UnsafeInMemorySorter( * Free the memory used by pointer array. */ public void free() { - consumer.freeArray(array); - array = null; + if (consumer != null) { + consumer.freeArray(array); + array = null; + } } public void reset() { From cd81fc9e8652c07b84f0887a24d67381b4e605fa Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 13 Jan 2016 16:34:23 -0800 Subject: [PATCH 101/110] [SPARK-12400][SHUFFLE] Avoid generating temp shuffle files for empty partitions This problem lies in `BypassMergeSortShuffleWriter`, empty partition will also generate a temp shuffle file with several bytes. So here change to only create file when partition is not empty. This problem only lies in here, no such issue in `HashShuffleWriter`. Please help to review, thanks a lot. Author: jerryshao Closes #10376 from jerryshao/SPARK-12400. --- .../sort/BypassMergeSortShuffleWriter.java | 25 ++++++------ .../BypassMergeSortShuffleWriterSuite.scala | 38 ++++++++++++++++++- 2 files changed, 51 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index a1a1fb01426a0..56cdc22f36261 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -138,7 +138,7 @@ public void write(Iterator> records) throws IOException { final File file = tempShuffleBlockIdPlusFile._2(); final BlockId blockId = tempShuffleBlockIdPlusFile._1(); partitionWriters[i] = - blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, writeMetrics).open(); + blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, writeMetrics); } // Creating the file to write to and creating a disk writer both involve interacting with // the disk, and can take a long time in aggregate when we open many files, so should be @@ -185,16 +185,19 @@ private long[] writePartitionedFile(File outputFile) throws IOException { boolean threwException = true; try { for (int i = 0; i < numPartitions; i++) { - final FileInputStream in = new FileInputStream(partitionWriters[i].fileSegment().file()); - boolean copyThrewException = true; - try { - lengths[i] = Utils.copyStream(in, out, false, transferToEnabled); - copyThrewException = false; - } finally { - Closeables.close(in, copyThrewException); - } - if (!partitionWriters[i].fileSegment().file().delete()) { - logger.error("Unable to delete file for partition {}", i); + final File file = partitionWriters[i].fileSegment().file(); + if (file.exists()) { + final FileInputStream in = new FileInputStream(file); + boolean copyThrewException = true; + try { + lengths[i] = Utils.copyStream(in, out, false, transferToEnabled); + copyThrewException = false; + } finally { + Closeables.close(in, copyThrewException); + } + if (!file.delete()) { + logger.error("Unable to delete file for partition {}", i); + } } } threwException = false; diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index e33408b94e2cf..ef6ce04e3ff28 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -105,7 +105,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte new Answer[(TempShuffleBlockId, File)] { override def answer(invocation: InvocationOnMock): (TempShuffleBlockId, File) = { val blockId = new TempShuffleBlockId(UUID.randomUUID) - val file = File.createTempFile(blockId.toString, null, tempDir) + val file = new File(tempDir, blockId.name) blockIdToFileMap.put(blockId, file) temporaryFilesCreated.append(file) (blockId, file) @@ -166,6 +166,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte writer.stop( /* success = */ true) assert(temporaryFilesCreated.nonEmpty) assert(writer.getPartitionLengths.sum === outputFile.length()) + assert(writer.getPartitionLengths.filter(_ == 0L).size === 4) // should be 4 zero length files assert(temporaryFilesCreated.count(_.exists()) === 0) // check that temporary files were deleted val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics.get assert(shuffleWriteMetrics.shuffleBytesWritten === outputFile.length()) @@ -174,6 +175,41 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte assert(taskMetrics.memoryBytesSpilled === 0) } + test("only generate temp shuffle file for non-empty partition") { + // Using exception to test whether only non-empty partition creates temp shuffle file, + // because temp shuffle file will only be cleaned after calling stop(false) in the failure + // case, so we could use it to validate the temp shuffle files. + def records: Iterator[(Int, Int)] = + Iterator((1, 1), (5, 5)) ++ + (0 until 100000).iterator.map { i => + if (i == 99990) { + throw new SparkException("intentional failure") + } else { + (2, 2) + } + } + + val writer = new BypassMergeSortShuffleWriter[Int, Int]( + blockManager, + blockResolver, + shuffleHandle, + 0, // MapId + taskContext, + conf + ) + + intercept[SparkException] { + writer.write(records) + } + + assert(temporaryFilesCreated.nonEmpty) + // Only 3 temp shuffle files will be created + assert(temporaryFilesCreated.count(_.exists()) === 3) + + writer.stop( /* success = */ false) + assert(temporaryFilesCreated.count(_.exists()) === 0) // check that temporary files were deleted + } + test("cleanup of intermediate files after errors") { val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, From 021dafc6a05a31dc22c9f9110dedb47a1f913087 Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Wed, 13 Jan 2016 17:43:27 -0800 Subject: [PATCH 102/110] [SPARK-12026][MLLIB] ChiSqTest gets slower and slower over time when number of features is large jira: https://issues.apache.org/jira/browse/SPARK-12026 The issue is valid as features.toArray.view.zipWithIndex.slice(startCol, endCol) becomes slower as startCol gets larger. I tested on local and the change can improve the performance and the running time was stable. Author: Yuhao Yang Closes #10146 from hhbyyh/chiSq. --- .../scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 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 f22f2df320f0d..4a3fb06469818 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 @@ -109,7 +109,9 @@ private[stat] object ChiSqTest extends Logging { } i += 1 distinctLabels += label - features.toArray.view.zipWithIndex.slice(startCol, endCol).map { case (feature, col) => + val brzFeatures = features.toBreeze + (startCol until endCol).map { col => + val feature = brzFeatures(col) allDistinctFeatures(col) += feature (col, feature, label) } @@ -122,7 +124,7 @@ private[stat] object ChiSqTest extends Logging { pairCounts.keys.filter(_._1 == startCol).map(_._3).toArray.distinct.zipWithIndex.toMap } val numLabels = labels.size - pairCounts.keys.groupBy(_._1).map { case (col, keys) => + pairCounts.keys.groupBy(_._1).foreach { 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)) From 20d8ef858af6e13db59df118b562ea33cba5464d Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 13 Jan 2016 18:01:29 -0800 Subject: [PATCH 103/110] [SPARK-12703][MLLIB][DOC][PYTHON] Fixed pyspark.mllib.clustering.KMeans user guide example Fixed WSSSE computeCost in Python mllib KMeans user guide example by using new computeCost method API in Python. Author: Joseph K. Bradley Closes #10707 from jkbradley/kmeans-doc-fix. --- docs/mllib-clustering.md | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 93cd0c1c61ae9..d0be03286849a 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -152,11 +152,7 @@ clusters = KMeans.train(parsedData, 2, maxIterations=10, runs=10, initializationMode="random") # Evaluate clustering by computing Within Set Sum of Squared Errors -def error(point): - center = clusters.centers[clusters.predict(point)] - return sqrt(sum([x**2 for x in (point - center)])) - -WSSSE = parsedData.map(lambda point: error(point)).reduce(lambda x, y: x + y) +WSSSE = clusters.computeCost(parsedData) print("Within Set Sum of Squared Error = " + str(WSSSE)) # Save and load model From e2ae7bd046f6d8d6a375c2e81e5a51d7d78ca984 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 13 Jan 2016 21:02:54 -0800 Subject: [PATCH 104/110] [SPARK-12819] Deprecate TaskContext.isRunningLocally() We've already removed local execution but didn't deprecate `TaskContext.isRunningLocally()`; we should deprecate it for 2.0. Author: Josh Rosen Closes #10751 from JoshRosen/remove-local-exec-from-taskcontext. --- core/src/main/scala/org/apache/spark/CacheManager.scala | 5 ----- core/src/main/scala/org/apache/spark/TaskContext.scala | 3 ++- .../main/scala/org/apache/spark/TaskContextImpl.scala | 3 +-- .../src/main/scala/org/apache/spark/scheduler/Task.scala | 3 +-- .../test/scala/org/apache/spark/CacheManagerSuite.scala | 9 --------- 5 files changed, 4 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 4d20c7369376e..36b536e89c3a4 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -68,11 +68,6 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { logInfo(s"Partition $key not found, computing it") val computedValues = rdd.computeOrReadCheckpoint(partition, context) - // If the task is running locally, do not persist the result - if (context.isRunningLocally) { - return computedValues - } - // Otherwise, cache the values and keep track of any updates in block statuses val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] val cachedValues = putInBlockManager(key, computedValues, storageLevel, updatedBlocks) diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index e25ed0fdd7fd2..7704abc134096 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -97,8 +97,9 @@ abstract class TaskContext extends Serializable { /** * Returns true if the task is running locally in the driver program. - * @return + * @return false */ + @deprecated("Local execution was removed, so this always returns false", "2.0.0") def isRunningLocally(): Boolean /** diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 6c493630997eb..94ff884b742b8 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -33,7 +33,6 @@ private[spark] class TaskContextImpl( override val taskMemoryManager: TaskMemoryManager, @transient private val metricsSystem: MetricsSystem, internalAccumulators: Seq[Accumulator[Long]], - val runningLocally: Boolean = false, val taskMetrics: TaskMetrics = TaskMetrics.empty) extends TaskContext with Logging { @@ -85,7 +84,7 @@ private[spark] class TaskContextImpl( override def isCompleted(): Boolean = completed - override def isRunningLocally(): Boolean = runningLocally + override def isRunningLocally(): Boolean = false override def isInterrupted(): Boolean = interrupted 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 0379ca2af6ab3..fca57928eca1b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -74,8 +74,7 @@ private[spark] abstract class Task[T]( attemptNumber, taskMemoryManager, metricsSystem, - internalAccumulators, - runningLocally = false) + internalAccumulators) TaskContext.setTaskContext(context) context.taskMetrics.setHostname(Utils.localHostName()) context.taskMetrics.setAccumulatorsUpdater(context.collectInternalAccumulators) diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index cb8bd04e496a7..30aa94c8a5971 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -82,15 +82,6 @@ class CacheManagerSuite extends SparkFunSuite with LocalSparkContext with Before assert(value.toList === List(5, 6, 7)) } - test("get uncached local rdd") { - // Local computation should not persist the resulting value, so don't expect a put(). - when(blockManager.get(RDDBlockId(0, 0))).thenReturn(None) - - val context = new TaskContextImpl(0, 0, 0, 0, null, null, Seq.empty, runningLocally = true) - val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) - assert(value.toList === List(1, 2, 3, 4)) - } - test("verify task metrics updated correctly") { cacheManager = sc.env.cacheManager val context = TaskContext.empty() From 962e9bcf94da6f5134983f2bf1e56c5cd84f2bf7 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 13 Jan 2016 22:43:28 -0800 Subject: [PATCH 105/110] [SPARK-12756][SQL] use hash expression in Exchange This PR makes bucketing and exchange share one common hash algorithm, so that we can guarantee the data distribution is same between shuffle and bucketed data source, which enables us to only shuffle one side when join a bucketed table and a normal one. This PR also fixes the tests that are broken by the new hash behaviour in shuffle. Author: Wenchen Fan Closes #10703 from cloud-fan/use-hash-expr-in-shuffle. --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- python/pyspark/sql/dataframe.py | 26 +++++++-------- python/pyspark/sql/group.py | 6 ++-- .../plans/physical/partitioning.scala | 7 +++- .../apache/spark/sql/execution/Exchange.scala | 12 +++++-- .../datasources/WriterContainer.scala | 20 +++++------ .../apache/spark/sql/JavaDataFrameSuite.java | 4 +-- .../apache/spark/sql/JavaDatasetSuite.java | 33 +++++++++++-------- .../org/apache/spark/sql/DataFrameSuite.scala | 21 +++++++----- .../org/apache/spark/sql/DatasetSuite.scala | 4 +-- .../org/apache/spark/sql/SQLQuerySuite.scala | 2 +- .../sql/sources/BucketedWriteSuite.scala | 11 ++++--- 12 files changed, 84 insertions(+), 64 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 97625b94a0e23..40d5066a93f4c 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1173,7 +1173,7 @@ test_that("group by, agg functions", { expect_equal(3, count(mean(gd))) expect_equal(3, count(max(gd))) - expect_equal(30, collect(max(gd))[1, 2]) + expect_equal(30, collect(max(gd))[2, 2]) expect_equal(1, collect(count(gd))[1, 2]) mockLines2 <- c("{\"name\":\"ID1\", \"value\": \"10\"}", diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index a7bc288e38861..90a6b5d9c0dda 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -403,10 +403,10 @@ def repartition(self, numPartitions, *cols): +---+-----+ |age| name| +---+-----+ - | 2|Alice| - | 2|Alice| | 5| Bob| | 5| Bob| + | 2|Alice| + | 2|Alice| +---+-----+ >>> data = data.repartition(7, "age") >>> data.show() @@ -552,7 +552,7 @@ def alias(self, alias): >>> df_as2 = df.alias("df_as2") >>> joined_df = df_as1.join(df_as2, col("df_as1.name") == col("df_as2.name"), 'inner') >>> joined_df.select(col("df_as1.name"), col("df_as2.name"), col("df_as2.age")).collect() - [Row(name=u'Alice', name=u'Alice', age=2), Row(name=u'Bob', name=u'Bob', age=5)] + [Row(name=u'Bob', name=u'Bob', age=5), Row(name=u'Alice', name=u'Alice', age=2)] """ assert isinstance(alias, basestring), "alias should be a string" return DataFrame(getattr(self._jdf, "as")(alias), self.sql_ctx) @@ -573,14 +573,14 @@ def join(self, other, on=None, how=None): One of `inner`, `outer`, `left_outer`, `right_outer`, `leftsemi`. >>> df.join(df2, df.name == df2.name, 'outer').select(df.name, df2.height).collect() - [Row(name=None, height=80), Row(name=u'Alice', height=None), Row(name=u'Bob', height=85)] + [Row(name=None, height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)] >>> df.join(df2, 'name', 'outer').select('name', 'height').collect() - [Row(name=u'Tom', height=80), Row(name=u'Alice', height=None), Row(name=u'Bob', height=85)] + [Row(name=u'Tom', height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)] >>> cond = [df.name == df3.name, df.age == df3.age] >>> df.join(df3, cond, 'outer').select(df.name, df3.age).collect() - [Row(name=u'Bob', age=5), Row(name=u'Alice', age=2)] + [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)] >>> df.join(df2, 'name').select(df.name, df2.height).collect() [Row(name=u'Bob', height=85)] @@ -880,9 +880,9 @@ def groupBy(self, *cols): >>> df.groupBy().avg().collect() [Row(avg(age)=3.5)] - >>> df.groupBy('name').agg({'age': 'mean'}).collect() + >>> sorted(df.groupBy('name').agg({'age': 'mean'}).collect()) [Row(name=u'Alice', avg(age)=2.0), Row(name=u'Bob', avg(age)=5.0)] - >>> df.groupBy(df.name).avg().collect() + >>> sorted(df.groupBy(df.name).avg().collect()) [Row(name=u'Alice', avg(age)=2.0), Row(name=u'Bob', avg(age)=5.0)] >>> df.groupBy(['name', df.age]).count().collect() [Row(name=u'Bob', age=5, count=1), Row(name=u'Alice', age=2, count=1)] @@ -901,11 +901,11 @@ def rollup(self, *cols): +-----+----+-----+ | name| age|count| +-----+----+-----+ - |Alice|null| 1| + |Alice| 2| 1| | Bob| 5| 1| | Bob|null| 1| | null|null| 2| - |Alice| 2| 1| + |Alice|null| 1| +-----+----+-----+ """ jgd = self._jdf.rollup(self._jcols(*cols)) @@ -923,12 +923,12 @@ def cube(self, *cols): | name| age|count| +-----+----+-----+ | null| 2| 1| - |Alice|null| 1| + |Alice| 2| 1| | Bob| 5| 1| - | Bob|null| 1| | null| 5| 1| + | Bob|null| 1| | null|null| 2| - |Alice| 2| 1| + |Alice|null| 1| +-----+----+-----+ """ jgd = self._jdf.cube(self._jcols(*cols)) diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index 9ca303a974cd4..ee734cb439287 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -74,11 +74,11 @@ def agg(self, *exprs): or a list of :class:`Column`. >>> gdf = df.groupBy(df.name) - >>> gdf.agg({"*": "count"}).collect() + >>> sorted(gdf.agg({"*": "count"}).collect()) [Row(name=u'Alice', count(1)=1), Row(name=u'Bob', count(1)=1)] >>> from pyspark.sql import functions as F - >>> gdf.agg(F.min(df.age)).collect() + >>> sorted(gdf.agg(F.min(df.age)).collect()) [Row(name=u'Alice', min(age)=2), Row(name=u'Bob', min(age)=5)] """ assert exprs, "exprs should not be empty" @@ -96,7 +96,7 @@ def agg(self, *exprs): def count(self): """Counts the number of records for each group. - >>> df.groupBy(df.age).count().collect() + >>> sorted(df.groupBy(df.age).count().collect()) [Row(age=2, count=1), Row(age=5, count=1)] """ 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 1bfe0ecb1e20b..d6e10c412ca1c 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.physical -import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder, Unevaluable} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{DataType, IntegerType} /** @@ -249,6 +249,11 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) case _ => false } + /** + * Returns an expression that will produce a valid partition ID(i.e. non-negative and is less + * than numPartitions) based on hashing expressions. + */ + def partitionIdExpression: Expression = Pmod(new Murmur3Hash(expressions), Literal(numPartitions)) } /** 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 058d147c7d65d..3770883af1e2f 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 @@ -143,7 +143,13 @@ case class Exchange( val rdd = child.execute() val part: Partitioner = newPartitioning match { case RoundRobinPartitioning(numPartitions) => new HashPartitioner(numPartitions) - case HashPartitioning(expressions, numPartitions) => new HashPartitioner(numPartitions) + case HashPartitioning(_, n) => + new Partitioner { + override def numPartitions: Int = n + // For HashPartitioning, the partitioning key is already a valid partition ID, as we use + // `HashPartitioning.partitionIdExpression` to produce partitioning key. + override def getPartition(key: Any): Int = key.asInstanceOf[Int] + } case RangePartitioning(sortingExpressions, numPartitions) => // Internally, RangePartitioner runs a job on the RDD that samples keys to compute // partition bounds. To get accurate samples, we need to copy the mutable keys. @@ -173,7 +179,9 @@ case class Exchange( position += 1 position } - case HashPartitioning(expressions, _) => newMutableProjection(expressions, child.output)() + case h: HashPartitioning => + val projection = UnsafeProjection.create(h.partitionIdExpression :: Nil, child.output) + row => projection(row).getInt(0) case RangePartitioning(_, _) | SinglePartition => identity case _ => sys.error(s"Exchange not implemented for $newPartitioning") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index fff72872c13b1..fc77529b7db32 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.execution.datasources import java.util.{Date, UUID} -import scala.collection.JavaConverters._ - import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter => MapReduceFileOutputCommitter} @@ -30,6 +28,7 @@ import org.apache.spark._ import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.UnsafeKVExternalSorter import org.apache.spark.sql.sources.{HadoopFsRelation, OutputWriter, OutputWriterFactory} @@ -322,9 +321,12 @@ private[sql] class DynamicPartitionWriterContainer( spec => spec.sortColumnNames.map(c => inputSchema.find(_.name == c).get) } - private def bucketIdExpression: Option[Expression] = for { - BucketSpec(numBuckets, _, _) <- bucketSpec - } yield Pmod(new Murmur3Hash(bucketColumns), Literal(numBuckets)) + private def bucketIdExpression: Option[Expression] = bucketSpec.map { spec => + // Use `HashPartitioning.partitionIdExpression` as our bucket id expression, so that we can + // guarantee the data distribution is same between shuffle and bucketed data source, which + // enables us to only shuffle one side when join a bucketed table and a normal one. + HashPartitioning(bucketColumns, spec.numBuckets).partitionIdExpression + } // Expressions that given a partition key build a string like: col1=val/col2=val/... private def partitionStringExpression: Seq[Expression] = { @@ -341,12 +343,8 @@ private[sql] class DynamicPartitionWriterContainer( } } - private def getBucketIdFromKey(key: InternalRow): Option[Int] = { - if (bucketSpec.isDefined) { - Some(key.getInt(partitionColumns.length)) - } else { - None - } + private def getBucketIdFromKey(key: InternalRow): Option[Int] = bucketSpec.map { _ => + key.getInt(partitionColumns.length) } /** diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index 8e0b2dbca4a98..ac1607ba3521a 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -237,8 +237,8 @@ public void testCrosstab() { DataFrame crosstab = df.stat().crosstab("a", "b"); String[] columnNames = crosstab.schema().fieldNames(); Assert.assertEquals("a_b", columnNames[0]); - Assert.assertEquals("1", columnNames[1]); - Assert.assertEquals("2", columnNames[2]); + Assert.assertEquals("2", columnNames[1]); + Assert.assertEquals("1", columnNames[2]); Row[] rows = crosstab.collect(); Arrays.sort(rows, crosstabRowComparator); Integer count = 1; diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java index 9f8db39e33d7e..1a3df1b117b68 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java @@ -187,7 +187,7 @@ public String call(Integer key, Iterator values) throws Exception { } }, Encoders.STRING()); - Assert.assertEquals(Arrays.asList("1a", "3foobar"), mapped.collectAsList()); + Assert.assertEquals(asSet("1a", "3foobar"), toSet(mapped.collectAsList())); Dataset flatMapped = grouped.flatMapGroups( new FlatMapGroupsFunction() { @@ -202,7 +202,7 @@ public Iterable call(Integer key, Iterator values) throws Except }, Encoders.STRING()); - Assert.assertEquals(Arrays.asList("1a", "3foobar"), flatMapped.collectAsList()); + Assert.assertEquals(asSet("1a", "3foobar"), toSet(flatMapped.collectAsList())); Dataset> reduced = grouped.reduce(new ReduceFunction() { @Override @@ -212,8 +212,8 @@ public String call(String v1, String v2) throws Exception { }); Assert.assertEquals( - Arrays.asList(tuple2(1, "a"), tuple2(3, "foobar")), - reduced.collectAsList()); + asSet(tuple2(1, "a"), tuple2(3, "foobar")), + toSet(reduced.collectAsList())); List data2 = Arrays.asList(2, 6, 10); Dataset ds2 = context.createDataset(data2, Encoders.INT()); @@ -245,7 +245,7 @@ public Iterable call( }, Encoders.STRING()); - Assert.assertEquals(Arrays.asList("1a#2", "3foobar#6", "5#10"), cogrouped.collectAsList()); + Assert.assertEquals(asSet("1a#2", "3foobar#6", "5#10"), toSet(cogrouped.collectAsList())); } @Test @@ -268,7 +268,7 @@ public String call(Integer key, Iterator data) throws Exception { }, Encoders.STRING()); - Assert.assertEquals(Arrays.asList("1a", "3foobar"), mapped.collectAsList()); + Assert.assertEquals(asSet("1a", "3foobar"), toSet(mapped.collectAsList())); } @Test @@ -290,9 +290,7 @@ public void testSetOperation() { List data = Arrays.asList("abc", "abc", "xyz"); Dataset ds = context.createDataset(data, Encoders.STRING()); - Assert.assertEquals( - Arrays.asList("abc", "xyz"), - sort(ds.distinct().collectAsList().toArray(new String[0]))); + Assert.assertEquals(asSet("abc", "xyz"), toSet(ds.distinct().collectAsList())); List data2 = Arrays.asList("xyz", "foo", "foo"); Dataset ds2 = context.createDataset(data2, Encoders.STRING()); @@ -302,16 +300,23 @@ public void testSetOperation() { Dataset unioned = ds.union(ds2); Assert.assertEquals( - Arrays.asList("abc", "abc", "foo", "foo", "xyz", "xyz"), - sort(unioned.collectAsList().toArray(new String[0]))); + Arrays.asList("abc", "abc", "xyz", "xyz", "foo", "foo"), + unioned.collectAsList()); Dataset subtracted = ds.subtract(ds2); Assert.assertEquals(Arrays.asList("abc", "abc"), subtracted.collectAsList()); } - private > List sort(T[] data) { - Arrays.sort(data); - return Arrays.asList(data); + private Set toSet(List records) { + Set set = new HashSet(); + for (T record : records) { + set.add(record); + } + return set; + } + + private Set asSet(T... records) { + return toSet(Arrays.asList(records)); } @Test diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 983dfbdedeefe..d6c140dfea9ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1083,17 +1083,20 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { // Walk each partition and verify that it is sorted descending and does not contain all // the values. df4.rdd.foreachPartition { p => - var previousValue: Int = -1 - var allSequential: Boolean = true - p.foreach { r => - val v: Int = r.getInt(1) - if (previousValue != -1) { - if (previousValue < v) throw new SparkException("Partition is not ordered.") - if (v + 1 != previousValue) allSequential = false + // Skip empty partition + if (p.hasNext) { + var previousValue: Int = -1 + var allSequential: Boolean = true + p.foreach { r => + val v: Int = r.getInt(1) + if (previousValue != -1) { + if (previousValue < v) throw new SparkException("Partition is not ordered.") + if (v + 1 != previousValue) allSequential = false + } + previousValue = v } - previousValue = v + if (allSequential) throw new SparkException("Partition should not be globally ordered") } - if (allSequential) throw new SparkException("Partition should not be globally ordered") } // Distribute and order by with multiple order bys diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 693f5aea2d015..d7b86e381108e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -456,8 +456,8 @@ class DatasetSuite extends QueryTest with SharedSQLContext { implicit val kryoEncoder = Encoders.kryo[KryoData] val ds = Seq(KryoData(1), KryoData(2)).toDS() - assert(ds.groupBy(p => p).count().collect().toSeq == - Seq((KryoData(1), 1L), (KryoData(2), 1L))) + assert(ds.groupBy(p => p).count().collect().toSet == + Set((KryoData(1), 1L), (KryoData(2), 1L))) } test("Kryo encoder self join") { 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 5de0979606b88..03d67c4e91f7f 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 @@ -806,7 +806,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC") .limit(2) .registerTempTable("subset1") - sql("SELECT DISTINCT n FROM lowerCaseData") + sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n ASC") .limit(2) .registerTempTable("subset2") checkAnswer( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala index b718b7cefb2a4..3ea9826544edb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala @@ -20,11 +20,11 @@ package org.apache.spark.sql.sources import java.io.File import org.apache.spark.sql.{AnalysisException, QueryTest} -import org.apache.spark.sql.catalyst.expressions.{Murmur3Hash, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.util.Utils class BucketedWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { import testImplicits._ @@ -98,11 +98,12 @@ class BucketedWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingle val qe = readBack.select(bucketCols.map(col): _*).queryExecution val rows = qe.toRdd.map(_.copy()).collect() - val getHashCode = - UnsafeProjection.create(new Murmur3Hash(qe.analyzed.output) :: Nil, qe.analyzed.output) + val getHashCode = UnsafeProjection.create( + HashPartitioning(qe.analyzed.output, 8).partitionIdExpression :: Nil, + qe.analyzed.output) for (row <- rows) { - val actualBucketId = Utils.nonNegativeMod(getHashCode(row).getInt(0), 8) + val actualBucketId = getHashCode(row).getInt(0) assert(actualBucketId == bucketId) } } From 8f13cd4cc8dcf638b178774418669a2e247d0652 Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Wed, 13 Jan 2016 23:50:08 -0800 Subject: [PATCH 106/110] =?UTF-8?q?[SPARK-12707][SPARK=20SUBMIT]=20Remove?= =?UTF-8?q?=20submit=20python/R=20scripts=20through=20py=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …spark/sparkR Author: Jeff Zhang Closes #10658 from zjffdu/SPARK-12707. --- .../spark/launcher/SparkSubmitCommandBuilder.java | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index a95f0f17517d1..269c89c310550 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -231,11 +231,9 @@ private List buildPySparkShellCommand(Map env) throws IO // the pyspark command line, then run it using spark-submit. if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".py")) { System.err.println( - "WARNING: Running python applications through 'pyspark' is deprecated as of Spark 1.0.\n" + + "Running python applications through 'pyspark' is not supported as of Spark 2.0.\n" + "Use ./bin/spark-submit "); - appResource = appArgs.get(0); - appArgs.remove(0); - return buildCommand(env); + System.exit(-1); } checkArgument(appArgs.isEmpty(), "pyspark does not support any application options."); @@ -258,9 +256,10 @@ private List buildPySparkShellCommand(Map env) throws IO private List buildSparkRCommand(Map env) throws IOException { if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".R")) { - appResource = appArgs.get(0); - appArgs.remove(0); - return buildCommand(env); + System.err.println( + "Running R applications through 'sparkR' is not supported as of Spark 2.0.\n" + + "Use ./bin/spark-submit "); + System.exit(-1); } // When launching the SparkR shell, store the spark-submit arguments in the SPARKR_SUBMIT_ARGS // env variable. From 56cdbd654d54bf07a063a03a5c34c4165818eeb2 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Thu, 14 Jan 2016 10:59:02 +0000 Subject: [PATCH 107/110] [SPARK-9844][CORE] File appender race condition during shutdown When an Executor process is destroyed, the FileAppender that is asynchronously reading the stderr stream of the process can throw an IOException during read because the stream is closed. Before the ExecutorRunner destroys the process, the FileAppender thread is flagged to stop. This PR wraps the inputStream.read call of the FileAppender in a try/catch block so that if an IOException is thrown and the thread has been flagged to stop, it will safely ignore the exception. Additionally, the FileAppender thread was changed to use Utils.tryWithSafeFinally to better log any exception that do occur. Added unit tests to verify a IOException is thrown and logged if FileAppender is not flagged to stop, and that no IOException when the flag is set. Author: Bryan Cutler Closes #10714 from BryanCutler/file-appender-read-ioexception-SPARK-9844. --- .../spark/util/logging/FileAppender.scala | 28 ++++--- .../apache/spark/util/FileAppenderSuite.scala | 77 +++++++++++++++++++ 2 files changed, 95 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala index 58c8560a3d049..86bbaa20f6cf2 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala @@ -17,7 +17,7 @@ package org.apache.spark.util.logging -import java.io.{File, FileOutputStream, InputStream} +import java.io.{File, FileOutputStream, InputStream, IOException} import org.apache.spark.{Logging, SparkConf} import org.apache.spark.util.{IntParam, Utils} @@ -58,20 +58,28 @@ private[spark] class FileAppender(inputStream: InputStream, file: File, bufferSi protected def appendStreamToFile() { try { logDebug("Started appending thread") - openFile() - val buf = new Array[Byte](bufferSize) - var n = 0 - while (!markedForStop && n != -1) { - n = inputStream.read(buf) - if (n != -1) { - appendToFile(buf, n) + Utils.tryWithSafeFinally { + openFile() + val buf = new Array[Byte](bufferSize) + var n = 0 + while (!markedForStop && n != -1) { + try { + n = inputStream.read(buf) + } catch { + // An InputStream can throw IOException during read if the stream is closed + // asynchronously, so once appender has been flagged to stop these will be ignored + case _: IOException if markedForStop => // do nothing and proceed to stop appending + } + if (n > 0) { + appendToFile(buf, n) + } } + } { + closeFile() } } catch { case e: Exception => logError(s"Error writing stream to file $file", e) - } finally { - closeFile() } } 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 98d1b28d5a167..b367cc8358342 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -18,12 +18,17 @@ package org.apache.spark.util import java.io._ +import java.util.concurrent.CountDownLatch import scala.collection.mutable.HashSet import scala.reflect._ import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files +import org.apache.log4j.{Appender, Level, Logger} +import org.apache.log4j.spi.LoggingEvent +import org.mockito.ArgumentCaptor +import org.mockito.Mockito.{atLeast, mock, verify} import org.scalatest.BeforeAndAfter import org.apache.spark.{Logging, SparkConf, SparkFunSuite} @@ -188,6 +193,67 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { testAppenderSelection[FileAppender, Any](rollingStrategy("xyz")) } + test("file appender async close stream abruptly") { + // Test FileAppender reaction to closing InputStream using a mock logging appender + val mockAppender = mock(classOf[Appender]) + val loggingEventCaptor = new ArgumentCaptor[LoggingEvent] + + // Make sure only logging errors + val logger = Logger.getRootLogger + logger.setLevel(Level.ERROR) + logger.addAppender(mockAppender) + + val testOutputStream = new PipedOutputStream() + val testInputStream = new PipedInputStream(testOutputStream) + + // Close the stream before appender tries to read will cause an IOException + testInputStream.close() + testOutputStream.close() + val appender = FileAppender(testInputStream, testFile, new SparkConf) + + appender.awaitTermination() + + // If InputStream was closed without first stopping the appender, an exception will be logged + verify(mockAppender, atLeast(1)).doAppend(loggingEventCaptor.capture) + val loggingEvent = loggingEventCaptor.getValue + assert(loggingEvent.getThrowableInformation !== null) + assert(loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException]) + } + + test("file appender async close stream gracefully") { + // Test FileAppender reaction to closing InputStream using a mock logging appender + val mockAppender = mock(classOf[Appender]) + val loggingEventCaptor = new ArgumentCaptor[LoggingEvent] + + // Make sure only logging errors + val logger = Logger.getRootLogger + logger.setLevel(Level.ERROR) + logger.addAppender(mockAppender) + + val testOutputStream = new PipedOutputStream() + val testInputStream = new PipedInputStream(testOutputStream) with LatchedInputStream + + // Close the stream before appender tries to read will cause an IOException + testInputStream.close() + testOutputStream.close() + val appender = FileAppender(testInputStream, testFile, new SparkConf) + + // Stop the appender before an IOException is called during read + testInputStream.latchReadStarted.await() + appender.stop() + testInputStream.latchReadProceed.countDown() + + appender.awaitTermination() + + // Make sure no IOException errors have been logged as a result of appender closing gracefully + verify(mockAppender, atLeast(0)).doAppend(loggingEventCaptor.capture) + import scala.collection.JavaConverters._ + loggingEventCaptor.getAllValues.asScala.foreach { loggingEvent => + assert(loggingEvent.getThrowableInformation === null + || !loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException]) + } + } + /** * Run the rolling file appender with data and see whether all the data was written correctly * across rolled over files. @@ -228,4 +294,15 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { file.getName.startsWith(testFile.getName) }.foreach { _.delete() } } + + /** Used to synchronize when read is called on a stream */ + private trait LatchedInputStream extends PipedInputStream { + val latchReadStarted = new CountDownLatch(1) + val latchReadProceed = new CountDownLatch(1) + abstract override def read(): Int = { + latchReadStarted.countDown() + latchReadProceed.await() + super.read() + } + } } From 501e99ef0fbd2f2165095548fe67a3447ccbfc91 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 14 Jan 2016 09:50:57 -0800 Subject: [PATCH 108/110] [SPARK-12784][UI] Fix Spark UI IndexOutOfBoundsException with dynamic allocation Add `listener.synchronized` to get `storageStatusList` and `execInfo` atomically. Author: Shixiong Zhu Closes #10728 from zsxwing/SPARK-12784. --- .../spark/status/api/v1/ExecutorListResource.scala | 10 +++++++--- .../org/apache/spark/ui/exec/ExecutorsPage.scala | 13 ++++++++++--- 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala index 8ad4656b4dada..3bdba922328c2 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala @@ -28,9 +28,13 @@ private[v1] class ExecutorListResource(ui: SparkUI) { @GET def executorList(): Seq[ExecutorSummary] = { val listener = ui.executorsListener - val storageStatusList = listener.storageStatusList - (0 until storageStatusList.size).map { statusId => - ExecutorsPage.getExecInfo(listener, statusId) + listener.synchronized { + // The follow codes should be protected by `listener` to make sure no executors will be + // removed before we query their status. See SPARK-12784. + val storageStatusList = listener.storageStatusList + (0 until storageStatusList.size).map { statusId => + ExecutorsPage.getExecInfo(listener, statusId) + } } } } 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 1a29b0f412603..7072a152d6b69 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 @@ -52,12 +52,19 @@ private[ui] class ExecutorsPage( private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - val storageStatusList = listener.storageStatusList + val (storageStatusList, execInfo) = listener.synchronized { + // The follow codes should be protected by `listener` to make sure no executors will be + // removed before we query their status. See SPARK-12784. + val _storageStatusList = listener.storageStatusList + val _execInfo = { + for (statusId <- 0 until _storageStatusList.size) + yield ExecutorsPage.getExecInfo(listener, statusId) + } + (_storageStatusList, _execInfo) + } 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 - ExecutorsPage.getExecInfo(listener, statusId) val execInfoSorted = execInfo.sortBy(_.id) val logsExist = execInfo.filter(_.executorLogs.nonEmpty).nonEmpty From 902667fd2766f0472a15851b1ed8fb5859593f97 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 14 Jan 2016 10:09:03 -0800 Subject: [PATCH 109/110] [SPARK-12771][SQL] Simplify CaseWhen code generation The generated code for CaseWhen uses a control variable "got" to make sure we do not evaluate more branches once a branch is true. Changing that to generate just simple "if / else" would be slightly more efficient. This closes #10737. Author: Reynold Xin Closes #10755 from rxin/SPARK-12771. --- .../expressions/conditionalExpressions.scala | 60 +++++++++++-------- 1 file changed, 35 insertions(+), 25 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 8cc7bc1da2fc3..83abbcdc61175 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -137,45 +137,55 @@ case class CaseWhen(branches: Seq[(Expression, Expression)], elseValue: Option[E } override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val got = ctx.freshName("got") - - val cases = branches.map { case (condition, value) => - val cond = condition.gen(ctx) - val res = value.gen(ctx) + // Generate code that looks like: + // + // condA = ... + // if (condA) { + // valueA + // } else { + // condB = ... + // if (condB) { + // valueB + // } else { + // condC = ... + // if (condC) { + // valueC + // } else { + // elseValue + // } + // } + // } + val cases = branches.map { case (condExpr, valueExpr) => + val cond = condExpr.gen(ctx) + val res = valueExpr.gen(ctx) s""" - if (!$got) { - ${cond.code} - if (!${cond.isNull} && ${cond.value}) { - $got = true; - ${res.code} - ${ev.isNull} = ${res.isNull}; - ${ev.value} = ${res.value}; - } + ${cond.code} + if (!${cond.isNull} && ${cond.value}) { + ${res.code} + ${ev.isNull} = ${res.isNull}; + ${ev.value} = ${res.value}; } """ - }.mkString("\n") + } - val elseCase = { - if (elseValue.isDefined) { - val res = elseValue.get.gen(ctx) + var generatedCode = cases.mkString("", "\nelse {\n", "\nelse {\n") + + elseValue.foreach { elseExpr => + val res = elseExpr.gen(ctx) + generatedCode += s""" - if (!$got) { ${res.code} ${ev.isNull} = ${res.isNull}; ${ev.value} = ${res.value}; - } """ - } else { - "" - } } + generatedCode += "}\n" * cases.size + s""" - boolean $got = false; boolean ${ev.isNull} = true; ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; - $cases - $elseCase + $generatedCode """ } From bcc7373f673d1a51b48fb95432ba5c4644dd5d23 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 14 Jan 2016 10:43:39 -0800 Subject: [PATCH 110/110] [SPARK-12821][BUILD] Style checker should run when some configuration files for style are modified but any source files are not. When running the `run-tests` script, style checkers run only when any source files are modified but they should run when configuration files related to style are modified. Author: Kousuke Saruta Closes #10754 from sarutak/SPARK-12821. --- dev/run-tests.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index 8726889cbc777..795db0dcfbab9 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -529,9 +529,14 @@ def main(): run_apache_rat_checks() # style checks - if not changed_files or any(f.endswith(".scala") for f in changed_files): + if not changed_files or any(f.endswith(".scala") + or f.endswith("scalastyle-config.xml") + for f in changed_files): run_scala_style_checks() - if not changed_files or any(f.endswith(".java") for f in changed_files): + if not changed_files or any(f.endswith(".java") + or f.endswith("checkstyle.xml") + or f.endswith("checkstyle-suppressions.xml") + for f in changed_files): # run_java_style_checks() pass if not changed_files or any(f.endswith(".py") for f in changed_files):
    Property NameDefaultMeaning
    spark.mesos.coarsefalsetrue - If set to true, runs over Mesos clusters in - "coarse-grained" sharing mode, - where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per - Spark task. This gives lower-latency scheduling for short queries, but leaves resources in use - for the whole duration of the Spark job. + If set to true, runs over Mesos clusters in "coarse-grained" sharing mode, where Spark acquires one long-lived Mesos task on each machine. + If set to false, runs over Mesos cluster in "fine-grained" sharing mode, where one Mesos task is created per Spark task. + Detailed information in 'Mesos Run Modes'.