From 49d767d838691fc7d964be2c4349662f5500ff2b Mon Sep 17 00:00:00 2001 From: actuaryzhang Date: Fri, 30 Jun 2017 20:02:15 +0800 Subject: [PATCH 001/125] [SPARK-18710][ML] Add offset in GLM ## What changes were proposed in this pull request? Add support for offset in GLM. This is useful for at least two reasons: 1. Account for exposure: e.g., when modeling the number of accidents, we may need to use miles driven as an offset to access factors on frequency. 2. Test incremental effects of new variables: we can use predictions from the existing model as offset and run a much smaller model on only new variables. This avoids re-estimating the large model with all variables (old + new) and can be very important for efficient large-scaled analysis. ## How was this patch tested? New test. yanboliang srowen felixcheung sethah Author: actuaryzhang Closes #16699 from actuaryzhang/offset. --- .../apache/spark/ml/feature/Instance.scala | 21 + .../IterativelyReweightedLeastSquares.scala | 14 +- .../spark/ml/optim/WeightedLeastSquares.scala | 2 +- .../GeneralizedLinearRegression.scala | 184 +++-- ...erativelyReweightedLeastSquaresSuite.scala | 40 +- .../GeneralizedLinearRegressionSuite.scala | 634 ++++++++++-------- 6 files changed, 534 insertions(+), 361 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala index cce3ca45ccd8..dd56fbbfa2b6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala @@ -27,3 +27,24 @@ import org.apache.spark.ml.linalg.Vector * @param features The vector of features for this data point. */ private[ml] case class Instance(label: Double, weight: Double, features: Vector) + +/** + * Case class that represents an instance of data point with + * label, weight, offset and features. + * This is mainly used in GeneralizedLinearRegression currently. + * + * @param label Label for this data point. + * @param weight The weight of this instance. + * @param offset The offset used for this data point. + * @param features The vector of features for this data point. + */ +private[ml] case class OffsetInstance( + label: Double, + weight: Double, + offset: Double, + features: Vector) { + + /** Converts to an [[Instance]] object by leaving out the offset. */ + def toInstance: Instance = Instance(label, weight, features) + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala index 9c495512422b..6961b45f55e4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.optim import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, OffsetInstance} import org.apache.spark.ml.linalg._ import org.apache.spark.rdd.RDD @@ -43,7 +43,7 @@ private[ml] class IterativelyReweightedLeastSquaresModel( * find M-estimator in robust regression and other optimization problems. * * @param initialModel the initial guess model. - * @param reweightFunc the reweight function which is used to update offsets and weights + * @param reweightFunc the reweight function which is used to update working labels and weights * at each iteration. * @param fitIntercept whether to fit intercept. * @param regParam L2 regularization parameter used by WLS. @@ -57,13 +57,13 @@ private[ml] class IterativelyReweightedLeastSquaresModel( */ private[ml] class IterativelyReweightedLeastSquares( val initialModel: WeightedLeastSquaresModel, - val reweightFunc: (Instance, WeightedLeastSquaresModel) => (Double, Double), + val reweightFunc: (OffsetInstance, WeightedLeastSquaresModel) => (Double, Double), val fitIntercept: Boolean, val regParam: Double, val maxIter: Int, val tol: Double) extends Logging with Serializable { - def fit(instances: RDD[Instance]): IterativelyReweightedLeastSquaresModel = { + def fit(instances: RDD[OffsetInstance]): IterativelyReweightedLeastSquaresModel = { var converged = false var iter = 0 @@ -75,10 +75,10 @@ private[ml] class IterativelyReweightedLeastSquares( oldModel = model - // Update offsets and weights using reweightFunc + // Update working labels and weights using reweightFunc val newInstances = instances.map { instance => - val (newOffset, newWeight) = reweightFunc(instance, oldModel) - Instance(newOffset, newWeight, instance.features) + val (newLabel, newWeight) = reweightFunc(instance, oldModel) + Instance(newLabel, newWeight, instance.features) } // Estimate new model diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala index 56ab9675700a..32b0af72ba9b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.optim import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, OffsetInstance} import org.apache.spark.ml.linalg._ import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index bff0d9bbb46f..ce3460ae4356 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -26,8 +26,8 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.PredictorParams -import org.apache.spark.ml.feature.Instance -import org.apache.spark.ml.linalg.{BLAS, Vector} +import org.apache.spark.ml.feature.{Instance, OffsetInstance} +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} import org.apache.spark.ml.optim._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ @@ -138,6 +138,27 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam @Since("2.0.0") def getLinkPredictionCol: String = $(linkPredictionCol) + /** + * Param for offset column name. If this is not set or empty, we treat all instance offsets + * as 0.0. The feature specified as offset has a constant coefficient of 1.0. + * @group param + */ + @Since("2.3.0") + final val offsetCol: Param[String] = new Param[String](this, "offsetCol", "The offset " + + "column name. If this is not set or empty, we treat all instance offsets as 0.0") + + /** @group getParam */ + @Since("2.3.0") + def getOffsetCol: String = $(offsetCol) + + /** Checks whether weight column is set and nonempty. */ + private[regression] def hasWeightCol: Boolean = + isSet(weightCol) && $(weightCol).nonEmpty + + /** Checks whether offset column is set and nonempty. */ + private[regression] def hasOffsetCol: Boolean = + isSet(offsetCol) && $(offsetCol).nonEmpty + /** Checks whether we should output link prediction. */ private[regression] def hasLinkPredictionCol: Boolean = { isDefined(linkPredictionCol) && $(linkPredictionCol).nonEmpty @@ -172,6 +193,11 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam } val newSchema = super.validateAndTransformSchema(schema, fitting, featuresDataType) + + if (hasOffsetCol) { + SchemaUtils.checkNumericType(schema, $(offsetCol)) + } + if (hasLinkPredictionCol) { SchemaUtils.appendColumn(newSchema, $(linkPredictionCol), DoubleType) } else { @@ -306,6 +332,16 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val @Since("2.0.0") def setWeightCol(value: String): this.type = set(weightCol, value) + /** + * Sets the value of param [[offsetCol]]. + * If this is not set or empty, we treat all instance offsets as 0.0. + * Default is not set, so all instances have offset 0.0. + * + * @group setParam + */ + @Since("2.3.0") + def setOffsetCol(value: String): this.type = set(offsetCol, value) + /** * Sets the solver algorithm used for optimization. * Currently only supports "irls" which is also the default solver. @@ -329,7 +365,7 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val val numFeatures = dataset.select(col($(featuresCol))).first().getAs[Vector](0).size val instr = Instrumentation.create(this, dataset) - instr.logParams(labelCol, featuresCol, weightCol, predictionCol, linkPredictionCol, + instr.logParams(labelCol, featuresCol, weightCol, offsetCol, predictionCol, linkPredictionCol, family, solver, fitIntercept, link, maxIter, regParam, tol) instr.logNumFeatures(numFeatures) @@ -343,15 +379,16 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val "GeneralizedLinearRegression was given data with 0 features, and with Param fitIntercept " + "set to false. To fit a model with 0 features, fitIntercept must be set to true." ) - val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) - val instances: RDD[Instance] = - dataset.select(col($(labelCol)), w, col($(featuresCol))).rdd.map { - case Row(label: Double, weight: Double, features: Vector) => - Instance(label, weight, features) - } + val w = if (!hasWeightCol) lit(1.0) else col($(weightCol)) + val offset = if (!hasOffsetCol) lit(0.0) else col($(offsetCol)).cast(DoubleType) val model = if (familyAndLink.family == Gaussian && familyAndLink.link == Identity) { // TODO: Make standardizeFeatures and standardizeLabel configurable. + val instances: RDD[Instance] = + dataset.select(col($(labelCol)), w, offset, col($(featuresCol))).rdd.map { + case Row(label: Double, weight: Double, offset: Double, features: Vector) => + Instance(label - offset, weight, features) + } val optimizer = new WeightedLeastSquares($(fitIntercept), $(regParam), elasticNetParam = 0.0, standardizeFeatures = true, standardizeLabel = true) val wlsModel = optimizer.fit(instances) @@ -362,6 +399,11 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val wlsModel.diagInvAtWA.toArray, 1, getSolver) model.setSummary(Some(trainingSummary)) } else { + val instances: RDD[OffsetInstance] = + dataset.select(col($(labelCol)), w, offset, col($(featuresCol))).rdd.map { + case Row(label: Double, weight: Double, offset: Double, features: Vector) => + OffsetInstance(label, weight, offset, features) + } // Fit Generalized Linear Model by iteratively reweighted least squares (IRLS). val initialModel = familyAndLink.initialize(instances, $(fitIntercept), $(regParam)) val optimizer = new IterativelyReweightedLeastSquares(initialModel, @@ -425,12 +467,12 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * Get the initial guess model for [[IterativelyReweightedLeastSquares]]. */ def initialize( - instances: RDD[Instance], + instances: RDD[OffsetInstance], fitIntercept: Boolean, regParam: Double): WeightedLeastSquaresModel = { val newInstances = instances.map { instance => val mu = family.initialize(instance.label, instance.weight) - val eta = predict(mu) + val eta = predict(mu) - instance.offset Instance(eta, instance.weight, instance.features) } // TODO: Make standardizeFeatures and standardizeLabel configurable. @@ -441,16 +483,16 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine } /** - * The reweight function used to update offsets and weights + * The reweight function used to update working labels and weights * at each iteration of [[IterativelyReweightedLeastSquares]]. */ - val reweightFunc: (Instance, WeightedLeastSquaresModel) => (Double, Double) = { - (instance: Instance, model: WeightedLeastSquaresModel) => { - val eta = model.predict(instance.features) + val reweightFunc: (OffsetInstance, WeightedLeastSquaresModel) => (Double, Double) = { + (instance: OffsetInstance, model: WeightedLeastSquaresModel) => { + val eta = model.predict(instance.features) + instance.offset val mu = fitted(eta) - val offset = eta + (instance.label - mu) * link.deriv(mu) - val weight = instance.weight / (math.pow(this.link.deriv(mu), 2.0) * family.variance(mu)) - (offset, weight) + val newLabel = eta - instance.offset + (instance.label - mu) * link.deriv(mu) + val newWeight = instance.weight / (math.pow(this.link.deriv(mu), 2.0) * family.variance(mu)) + (newLabel, newWeight) } } } @@ -950,15 +992,22 @@ class GeneralizedLinearRegressionModel private[ml] ( private lazy val familyAndLink = FamilyAndLink(this) override protected def predict(features: Vector): Double = { - val eta = predictLink(features) + predict(features, 0.0) + } + + /** + * Calculates the predicted value when offset is set. + */ + private def predict(features: Vector, offset: Double): Double = { + val eta = predictLink(features, offset) familyAndLink.fitted(eta) } /** - * Calculate the link prediction (linear predictor) of the given instance. + * Calculates the link prediction (linear predictor) of the given instance. */ - private def predictLink(features: Vector): Double = { - BLAS.dot(features, coefficients) + intercept + private def predictLink(features: Vector, offset: Double): Double = { + BLAS.dot(features, coefficients) + intercept + offset } override def transform(dataset: Dataset[_]): DataFrame = { @@ -967,14 +1016,16 @@ class GeneralizedLinearRegressionModel private[ml] ( } override protected def transformImpl(dataset: Dataset[_]): DataFrame = { - val predictUDF = udf { (features: Vector) => predict(features) } - val predictLinkUDF = udf { (features: Vector) => predictLink(features) } + val predictUDF = udf { (features: Vector, offset: Double) => predict(features, offset) } + val predictLinkUDF = udf { (features: Vector, offset: Double) => predictLink(features, offset) } + + val offset = if (!hasOffsetCol) lit(0.0) else col($(offsetCol)).cast(DoubleType) var output = dataset if ($(predictionCol).nonEmpty) { - output = output.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) + output = output.withColumn($(predictionCol), predictUDF(col($(featuresCol)), offset)) } if (hasLinkPredictionCol) { - output = output.withColumn($(linkPredictionCol), predictLinkUDF(col($(featuresCol)))) + output = output.withColumn($(linkPredictionCol), predictLinkUDF(col($(featuresCol)), offset)) } output.toDF() } @@ -1146,9 +1197,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( /** Degrees of freedom. */ @Since("2.0.0") - lazy val degreesOfFreedom: Long = { - numInstances - rank - } + lazy val degreesOfFreedom: Long = numInstances - rank /** The residual degrees of freedom. */ @Since("2.0.0") @@ -1156,18 +1205,20 @@ class GeneralizedLinearRegressionSummary private[regression] ( /** The residual degrees of freedom for the null model. */ @Since("2.0.0") - lazy val residualDegreeOfFreedomNull: Long = if (model.getFitIntercept) { - numInstances - 1 - } else { - numInstances + lazy val residualDegreeOfFreedomNull: Long = { + if (model.getFitIntercept) numInstances - 1 else numInstances } - private def weightCol: Column = { - if (!model.isDefined(model.weightCol) || model.getWeightCol.isEmpty) { - lit(1.0) - } else { - col(model.getWeightCol) - } + private def label: Column = col(model.getLabelCol).cast(DoubleType) + + private def prediction: Column = col(predictionCol) + + private def weight: Column = { + if (!model.hasWeightCol) lit(1.0) else col(model.getWeightCol) + } + + private def offset: Column = { + if (!model.hasOffsetCol) lit(0.0) else col(model.getOffsetCol).cast(DoubleType) } private[regression] lazy val devianceResiduals: DataFrame = { @@ -1175,25 +1226,23 @@ class GeneralizedLinearRegressionSummary private[regression] ( val r = math.sqrt(math.max(family.deviance(y, mu, weight), 0.0)) if (y > mu) r else -1.0 * r } - val w = weightCol predictions.select( - drUDF(col(model.getLabelCol), col(predictionCol), w).as("devianceResiduals")) + drUDF(label, prediction, weight).as("devianceResiduals")) } private[regression] lazy val pearsonResiduals: DataFrame = { val prUDF = udf { mu: Double => family.variance(mu) } - val w = weightCol - predictions.select(col(model.getLabelCol).minus(col(predictionCol)) - .multiply(sqrt(w)).divide(sqrt(prUDF(col(predictionCol)))).as("pearsonResiduals")) + predictions.select(label.minus(prediction) + .multiply(sqrt(weight)).divide(sqrt(prUDF(prediction))).as("pearsonResiduals")) } private[regression] lazy val workingResiduals: DataFrame = { val wrUDF = udf { (y: Double, mu: Double) => (y - mu) * link.deriv(mu) } - predictions.select(wrUDF(col(model.getLabelCol), col(predictionCol)).as("workingResiduals")) + predictions.select(wrUDF(label, prediction).as("workingResiduals")) } private[regression] lazy val responseResiduals: DataFrame = { - predictions.select(col(model.getLabelCol).minus(col(predictionCol)).as("responseResiduals")) + predictions.select(label.minus(prediction).as("responseResiduals")) } /** @@ -1225,16 +1274,35 @@ class GeneralizedLinearRegressionSummary private[regression] ( */ @Since("2.0.0") lazy val nullDeviance: Double = { - val w = weightCol - val wtdmu: Double = if (model.getFitIntercept) { - val agg = predictions.agg(sum(w.multiply(col(model.getLabelCol))), sum(w)).first() - agg.getDouble(0) / agg.getDouble(1) + val intercept: Double = if (!model.getFitIntercept) { + 0.0 } else { - link.unlink(0.0) + /* + Estimate intercept analytically when there is no offset, or when there is offset but + the model is Gaussian family with identity link. Otherwise, fit an intercept only model. + */ + if (!model.hasOffsetCol || + (model.hasOffsetCol && family == Gaussian && link == Identity)) { + val agg = predictions.agg(sum(weight.multiply( + label.minus(offset))), sum(weight)).first() + link.link(agg.getDouble(0) / agg.getDouble(1)) + } else { + // Create empty feature column and fit intercept only model using param setting from model + val featureNull = "feature_" + java.util.UUID.randomUUID.toString + val paramMap = model.extractParamMap() + paramMap.put(model.featuresCol, featureNull) + if (family.name != "tweedie") { + paramMap.remove(model.variancePower) + } + val emptyVectorUDF = udf{ () => Vectors.zeros(0) } + model.parent.fit( + dataset.withColumn(featureNull, emptyVectorUDF()), paramMap + ).intercept + } } - predictions.select(col(model.getLabelCol).cast(DoubleType), w).rdd.map { - case Row(y: Double, weight: Double) => - family.deviance(y, wtdmu, weight) + predictions.select(label, offset, weight).rdd.map { + case Row(y: Double, offset: Double, weight: Double) => + family.deviance(y, link.unlink(intercept + offset), weight) }.sum() } @@ -1243,8 +1311,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( */ @Since("2.0.0") lazy val deviance: Double = { - val w = weightCol - predictions.select(col(model.getLabelCol).cast(DoubleType), col(predictionCol), w).rdd.map { + predictions.select(label, prediction, weight).rdd.map { case Row(label: Double, pred: Double, weight: Double) => family.deviance(label, pred, weight) }.sum() @@ -1269,10 +1336,9 @@ class GeneralizedLinearRegressionSummary private[regression] ( /** Akaike Information Criterion (AIC) for the fitted model. */ @Since("2.0.0") lazy val aic: Double = { - val w = weightCol - val weightSum = predictions.select(w).agg(sum(w)).first().getDouble(0) + val weightSum = predictions.select(weight).agg(sum(weight)).first().getDouble(0) val t = predictions.select( - col(model.getLabelCol).cast(DoubleType), col(predictionCol), w).rdd.map { + label, prediction, weight).rdd.map { case Row(label: Double, pred: Double, weight: Double) => (label, pred, weight) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala index 50260952ecb6..6d143504fcf5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.optim import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, OffsetInstance} import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -26,8 +26,8 @@ import org.apache.spark.rdd.RDD class IterativelyReweightedLeastSquaresSuite extends SparkFunSuite with MLlibTestSparkContext { - private var instances1: RDD[Instance] = _ - private var instances2: RDD[Instance] = _ + private var instances1: RDD[OffsetInstance] = _ + private var instances2: RDD[OffsetInstance] = _ override def beforeAll(): Unit = { super.beforeAll() @@ -39,10 +39,10 @@ class IterativelyReweightedLeastSquaresSuite extends SparkFunSuite with MLlibTes w <- c(1, 2, 3, 4) */ instances1 = sc.parallelize(Seq( - Instance(1.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), - Instance(0.0, 2.0, Vectors.dense(1.0, 2.0)), - Instance(1.0, 3.0, Vectors.dense(2.0, 1.0)), - Instance(0.0, 4.0, Vectors.dense(3.0, 3.0)) + OffsetInstance(1.0, 1.0, 0.0, Vectors.dense(0.0, 5.0).toSparse), + OffsetInstance(0.0, 2.0, 0.0, Vectors.dense(1.0, 2.0)), + OffsetInstance(1.0, 3.0, 0.0, Vectors.dense(2.0, 1.0)), + OffsetInstance(0.0, 4.0, 0.0, Vectors.dense(3.0, 3.0)) ), 2) /* R code: @@ -52,10 +52,10 @@ class IterativelyReweightedLeastSquaresSuite extends SparkFunSuite with MLlibTes w <- c(1, 2, 3, 4) */ instances2 = sc.parallelize(Seq( - Instance(2.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), - Instance(8.0, 2.0, Vectors.dense(1.0, 7.0)), - Instance(3.0, 3.0, Vectors.dense(2.0, 11.0)), - Instance(9.0, 4.0, Vectors.dense(3.0, 13.0)) + OffsetInstance(2.0, 1.0, 0.0, Vectors.dense(0.0, 5.0).toSparse), + OffsetInstance(8.0, 2.0, 0.0, Vectors.dense(1.0, 7.0)), + OffsetInstance(3.0, 3.0, 0.0, Vectors.dense(2.0, 11.0)), + OffsetInstance(9.0, 4.0, 0.0, Vectors.dense(3.0, 13.0)) ), 2) } @@ -156,7 +156,7 @@ class IterativelyReweightedLeastSquaresSuite extends SparkFunSuite with MLlibTes var idx = 0 for (fitIntercept <- Seq(false, true)) { val initial = new WeightedLeastSquares(fitIntercept, regParam = 0.0, elasticNetParam = 0.0, - standardizeFeatures = false, standardizeLabel = false).fit(instances2) + standardizeFeatures = false, standardizeLabel = false).fit(instances2.map(_.toInstance)) val irls = new IterativelyReweightedLeastSquares(initial, L1RegressionReweightFunc, fitIntercept, regParam = 0.0, maxIter = 200, tol = 1e-7).fit(instances2) val actual = Vectors.dense(irls.intercept, irls.coefficients(0), irls.coefficients(1)) @@ -169,29 +169,29 @@ class IterativelyReweightedLeastSquaresSuite extends SparkFunSuite with MLlibTes object IterativelyReweightedLeastSquaresSuite { def BinomialReweightFunc( - instance: Instance, + instance: OffsetInstance, model: WeightedLeastSquaresModel): (Double, Double) = { - val eta = model.predict(instance.features) + val eta = model.predict(instance.features) + instance.offset val mu = 1.0 / (1.0 + math.exp(-1.0 * eta)) - val z = eta + (instance.label - mu) / (mu * (1.0 - mu)) + val z = eta - instance.offset + (instance.label - mu) / (mu * (1.0 - mu)) val w = mu * (1 - mu) * instance.weight (z, w) } def PoissonReweightFunc( - instance: Instance, + instance: OffsetInstance, model: WeightedLeastSquaresModel): (Double, Double) = { - val eta = model.predict(instance.features) + val eta = model.predict(instance.features) + instance.offset val mu = math.exp(eta) - val z = eta + (instance.label - mu) / mu + val z = eta - instance.offset + (instance.label - mu) / mu val w = mu * instance.weight (z, w) } def L1RegressionReweightFunc( - instance: Instance, + instance: OffsetInstance, model: WeightedLeastSquaresModel): (Double, Double) = { - val eta = model.predict(instance.features) + val eta = model.predict(instance.features) + instance.offset val e = math.max(math.abs(eta - instance.label), 1e-7) val w = 1 / e val y = instance.label diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala index f7c7c001a36a..cfaa57314bd6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala @@ -21,7 +21,7 @@ import scala.util.Random import org.apache.spark.SparkFunSuite import org.apache.spark.ml.classification.LogisticRegressionSuite._ -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, OffsetInstance} import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors} import org.apache.spark.ml.param.{ParamMap, ParamsSuite} @@ -797,77 +797,160 @@ class GeneralizedLinearRegressionSuite } } - test("glm summary: gaussian family with weight") { + test("generalized linear regression with weight and offset") { /* - R code: + R code: + library(statmod) - A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) - b <- c(17, 19, 23, 29) - w <- c(1, 2, 3, 4) - df <- as.data.frame(cbind(A, b)) - */ - val datasetWithWeight = Seq( - Instance(17.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), - Instance(19.0, 2.0, Vectors.dense(1.0, 7.0)), - Instance(23.0, 3.0, Vectors.dense(2.0, 11.0)), - Instance(29.0, 4.0, Vectors.dense(3.0, 13.0)) + df <- as.data.frame(matrix(c( + 0.2, 1.0, 2.0, 0.0, 5.0, + 0.5, 2.1, 0.5, 1.0, 2.0, + 0.9, 0.4, 1.0, 2.0, 1.0, + 0.7, 0.7, 0.0, 3.0, 3.0), 4, 5, byrow = TRUE)) + families <- list(gaussian, binomial, poisson, Gamma, tweedie(1.5)) + f1 <- V1 ~ -1 + V4 + V5 + f2 <- V1 ~ V4 + V5 + for (f in c(f1, f2)) { + for (fam in families) { + model <- glm(f, df, family = fam, weights = V2, offset = V3) + print(as.vector(coef(model))) + } + } + [1] 0.5169222 -0.3344444 + [1] 0.9419107 -0.6864404 + [1] 0.1812436 -0.6568422 + [1] -0.2869094 0.7857710 + [1] 0.1055254 0.2979113 + [1] -0.05990345 0.53188982 -0.32118415 + [1] -0.2147117 0.9911750 -0.6356096 + [1] -1.5616130 0.6646470 -0.3192581 + [1] 0.3390397 -0.3406099 0.6870259 + [1] 0.3665034 0.1039416 0.1484616 + */ + val dataset = Seq( + OffsetInstance(0.2, 1.0, 2.0, Vectors.dense(0.0, 5.0)), + OffsetInstance(0.5, 2.1, 0.5, Vectors.dense(1.0, 2.0)), + OffsetInstance(0.9, 0.4, 1.0, Vectors.dense(2.0, 1.0)), + OffsetInstance(0.7, 0.7, 0.0, Vectors.dense(3.0, 3.0)) ).toDF() + + val expected = Seq( + Vectors.dense(0, 0.5169222, -0.3344444), + Vectors.dense(0, 0.9419107, -0.6864404), + Vectors.dense(0, 0.1812436, -0.6568422), + Vectors.dense(0, -0.2869094, 0.785771), + Vectors.dense(0, 0.1055254, 0.2979113), + Vectors.dense(-0.05990345, 0.53188982, -0.32118415), + Vectors.dense(-0.2147117, 0.991175, -0.6356096), + Vectors.dense(-1.561613, 0.664647, -0.3192581), + Vectors.dense(0.3390397, -0.3406099, 0.6870259), + Vectors.dense(0.3665034, 0.1039416, 0.1484616)) + + import GeneralizedLinearRegression._ + + var idx = 0 + + for (fitIntercept <- Seq(false, true)) { + for (family <- Seq("gaussian", "binomial", "poisson", "gamma", "tweedie")) { + val trainer = new GeneralizedLinearRegression().setFamily(family) + .setFitIntercept(fitIntercept).setOffsetCol("offset") + .setWeightCol("weight").setLinkPredictionCol("linkPrediction") + if (family == "tweedie") trainer.setVariancePower(1.5) + val model = trainer.fit(dataset) + val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) + assert(actual ~= expected(idx) absTol 1e-4, s"Model mismatch: GLM with family = $family," + + s" and fitIntercept = $fitIntercept.") + + val familyLink = FamilyAndLink(trainer) + model.transform(dataset).select("features", "offset", "prediction", "linkPrediction") + .collect().foreach { + case Row(features: DenseVector, offset: Double, prediction1: Double, + linkPrediction1: Double) => + val eta = BLAS.dot(features, model.coefficients) + model.intercept + offset + val prediction2 = familyLink.fitted(eta) + val linkPrediction2 = eta + assert(prediction1 ~= prediction2 relTol 1E-5, "Prediction mismatch: GLM with " + + s"family = $family, and fitIntercept = $fitIntercept.") + assert(linkPrediction1 ~= linkPrediction2 relTol 1E-5, "Link Prediction mismatch: " + + s"GLM with family = $family, and fitIntercept = $fitIntercept.") + } + + idx += 1 + } + } + } + + test("glm summary: gaussian family with weight and offset") { /* - R code: + R code: - model <- glm(formula = "b ~ .", family="gaussian", data = df, weights = w) - summary(model) + A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) + b <- c(17, 19, 23, 29) + w <- c(1, 2, 3, 4) + off <- c(2, 3, 1, 4) + df <- as.data.frame(cbind(A, b)) + */ + val dataset = Seq( + OffsetInstance(17.0, 1.0, 2.0, Vectors.dense(0.0, 5.0).toSparse), + OffsetInstance(19.0, 2.0, 3.0, Vectors.dense(1.0, 7.0)), + OffsetInstance(23.0, 3.0, 1.0, Vectors.dense(2.0, 11.0)), + OffsetInstance(29.0, 4.0, 4.0, Vectors.dense(3.0, 13.0)) + ).toDF() + /* + R code: - Deviance Residuals: - 1 2 3 4 - 1.920 -1.358 -1.109 0.960 + model <- glm(formula = "b ~ .", family = "gaussian", data = df, + weights = w, offset = off) + summary(model) - Coefficients: - Estimate Std. Error t value Pr(>|t|) - (Intercept) 18.080 9.608 1.882 0.311 - V1 6.080 5.556 1.094 0.471 - V2 -0.600 1.960 -0.306 0.811 + Deviance Residuals: + 1 2 3 4 + 0.9600 -0.6788 -0.5543 0.4800 - (Dispersion parameter for gaussian family taken to be 7.68) + Coefficients: + Estimate Std. Error t value Pr(>|t|) + (Intercept) 5.5400 4.8040 1.153 0.455 + V1 -0.9600 2.7782 -0.346 0.788 + V2 1.7000 0.9798 1.735 0.333 - Null deviance: 202.00 on 3 degrees of freedom - Residual deviance: 7.68 on 1 degrees of freedom - AIC: 18.783 + (Dispersion parameter for gaussian family taken to be 1.92) - Number of Fisher Scoring iterations: 2 + Null deviance: 152.10 on 3 degrees of freedom + Residual deviance: 1.92 on 1 degrees of freedom + AIC: 13.238 - residuals(model, type="pearson") - 1 2 3 4 - 1.920000 -1.357645 -1.108513 0.960000 + Number of Fisher Scoring iterations: 2 - residuals(model, type="working") + residuals(model, type = "pearson") + 1 2 3 4 + 0.9600000 -0.6788225 -0.5542563 0.4800000 + residuals(model, type = "working") 1 2 3 4 - 1.92 -0.96 -0.64 0.48 - - residuals(model, type="response") + 0.96 -0.48 -0.32 0.24 + residuals(model, type = "response") 1 2 3 4 - 1.92 -0.96 -0.64 0.48 + 0.96 -0.48 -0.32 0.24 */ val trainer = new GeneralizedLinearRegression() - .setWeightCol("weight") + .setWeightCol("weight").setOffsetCol("offset") + + val model = trainer.fit(dataset) - val model = trainer.fit(datasetWithWeight) - - val coefficientsR = Vectors.dense(Array(6.080, -0.600)) - val interceptR = 18.080 - val devianceResidualsR = Array(1.920, -1.358, -1.109, 0.960) - val pearsonResidualsR = Array(1.920000, -1.357645, -1.108513, 0.960000) - val workingResidualsR = Array(1.92, -0.96, -0.64, 0.48) - val responseResidualsR = Array(1.92, -0.96, -0.64, 0.48) - val seCoefR = Array(5.556, 1.960, 9.608) - val tValsR = Array(1.094, -0.306, 1.882) - val pValsR = Array(0.471, 0.811, 0.311) - val dispersionR = 7.68 - val nullDevianceR = 202.00 - val residualDevianceR = 7.68 + val coefficientsR = Vectors.dense(Array(-0.96, 1.7)) + val interceptR = 5.54 + val devianceResidualsR = Array(0.96, -0.67882, -0.55426, 0.48) + val pearsonResidualsR = Array(0.96, -0.67882, -0.55426, 0.48) + val workingResidualsR = Array(0.96, -0.48, -0.32, 0.24) + val responseResidualsR = Array(0.96, -0.48, -0.32, 0.24) + val seCoefR = Array(2.7782, 0.9798, 4.804) + val tValsR = Array(-0.34555, 1.73506, 1.15321) + val pValsR = Array(0.78819, 0.33286, 0.45478) + val dispersionR = 1.92 + val nullDevianceR = 152.1 + val residualDevianceR = 1.92 val residualDegreeOfFreedomNullR = 3 val residualDegreeOfFreedomR = 1 - val aicR = 18.783 + val aicR = 13.23758 assert(model.hasSummary) val summary = model.summary @@ -912,7 +995,7 @@ class GeneralizedLinearRegressionSuite assert(summary.aic ~== aicR absTol 1E-3) assert(summary.solver === "irls") - val summary2: GeneralizedLinearRegressionSummary = model.evaluate(datasetWithWeight) + val summary2: GeneralizedLinearRegressionSummary = model.evaluate(dataset) assert(summary.predictions.columns.toSet === summary2.predictions.columns.toSet) assert(summary.predictionCol === summary2.predictionCol) assert(summary.rank === summary2.rank) @@ -925,79 +1008,79 @@ class GeneralizedLinearRegressionSuite assert(summary.aic === summary2.aic) } - test("glm summary: binomial family with weight") { + test("glm summary: binomial family with weight and offset") { /* - R code: + R code: - A <- matrix(c(0, 1, 2, 3, 5, 2, 1, 3), 4, 2) - b <- c(1, 0.5, 1, 0) - w <- c(1, 2.0, 0.3, 4.7) - df <- as.data.frame(cbind(A, b)) + df <- as.data.frame(matrix(c( + 0.2, 1.0, 2.0, 0.0, 5.0, + 0.5, 2.1, 0.5, 1.0, 2.0, + 0.9, 0.4, 1.0, 2.0, 1.0, + 0.7, 0.7, 0.0, 3.0, 3.0), 4, 5, byrow = TRUE)) */ - val datasetWithWeight = Seq( - Instance(1.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), - Instance(0.5, 2.0, Vectors.dense(1.0, 2.0)), - Instance(1.0, 0.3, Vectors.dense(2.0, 1.0)), - Instance(0.0, 4.7, Vectors.dense(3.0, 3.0)) + val dataset = Seq( + OffsetInstance(0.2, 1.0, 2.0, Vectors.dense(0.0, 5.0)), + OffsetInstance(0.5, 2.1, 0.5, Vectors.dense(1.0, 2.0)), + OffsetInstance(0.9, 0.4, 1.0, Vectors.dense(2.0, 1.0)), + OffsetInstance(0.7, 0.7, 0.0, Vectors.dense(3.0, 3.0)) ).toDF() - /* - R code: - - model <- glm(formula = "b ~ . -1", family="binomial", data = df, weights = w) - summary(model) - - Deviance Residuals: - 1 2 3 4 - 0.2404 0.1965 1.2824 -0.6916 + R code: - Coefficients: - Estimate Std. Error z value Pr(>|z|) - x1 -1.6901 1.2764 -1.324 0.185 - x2 0.7059 0.9449 0.747 0.455 + model <- glm(formula = "V1 ~ V4 + V5", family = "binomial", data = df, + weights = V2, offset = V3) + summary(model) - (Dispersion parameter for binomial family taken to be 1) + Deviance Residuals: + 1 2 3 4 + 0.002584 -0.003800 0.012478 -0.001796 - Null deviance: 8.3178 on 4 degrees of freedom - Residual deviance: 2.2193 on 2 degrees of freedom - AIC: 5.9915 + Coefficients: + Estimate Std. Error z value Pr(>|z|) + (Intercept) -0.2147 3.5687 -0.060 0.952 + V4 0.9912 1.2344 0.803 0.422 + V5 -0.6356 0.9669 -0.657 0.511 - Number of Fisher Scoring iterations: 5 + (Dispersion parameter for binomial family taken to be 1) - residuals(model, type="pearson") - 1 2 3 4 - 0.171217 0.197406 2.085864 -0.495332 + Null deviance: 2.17560881 on 3 degrees of freedom + Residual deviance: 0.00018005 on 1 degrees of freedom + AIC: 10.245 - residuals(model, type="working") - 1 2 3 4 - 1.029315 0.281881 15.502768 -1.052203 + Number of Fisher Scoring iterations: 4 - residuals(model, type="response") - 1 2 3 4 - 0.028480 0.069123 0.935495 -0.049613 + residuals(model, type = "pearson") + 1 2 3 4 + 0.002586113 -0.003799744 0.012372235 -0.001796892 + residuals(model, type = "working") + 1 2 3 4 + 0.006477857 -0.005244163 0.063541250 -0.004691064 + residuals(model, type = "response") + 1 2 3 4 + 0.0010324375 -0.0013110318 0.0060225522 -0.0009832738 */ val trainer = new GeneralizedLinearRegression() .setFamily("Binomial") .setWeightCol("weight") - .setFitIntercept(false) - - val model = trainer.fit(datasetWithWeight) - - val coefficientsR = Vectors.dense(Array(-1.690134, 0.705929)) - val interceptR = 0.0 - val devianceResidualsR = Array(0.2404, 0.1965, 1.2824, -0.6916) - val pearsonResidualsR = Array(0.171217, 0.197406, 2.085864, -0.495332) - val workingResidualsR = Array(1.029315, 0.281881, 15.502768, -1.052203) - val responseResidualsR = Array(0.02848, 0.069123, 0.935495, -0.049613) - val seCoefR = Array(1.276417, 0.944934) - val tValsR = Array(-1.324124, 0.747068) - val pValsR = Array(0.185462, 0.455023) - val dispersionR = 1.0 - val nullDevianceR = 8.3178 - val residualDevianceR = 2.2193 - val residualDegreeOfFreedomNullR = 4 - val residualDegreeOfFreedomR = 2 - val aicR = 5.991537 + .setOffsetCol("offset") + + val model = trainer.fit(dataset) + + val coefficientsR = Vectors.dense(Array(0.99117, -0.63561)) + val interceptR = -0.21471 + val devianceResidualsR = Array(0.00258, -0.0038, 0.01248, -0.0018) + val pearsonResidualsR = Array(0.00259, -0.0038, 0.01237, -0.0018) + val workingResidualsR = Array(0.00648, -0.00524, 0.06354, -0.00469) + val responseResidualsR = Array(0.00103, -0.00131, 0.00602, -0.00098) + val seCoefR = Array(1.23439, 0.9669, 3.56866) + val tValsR = Array(0.80297, -0.65737, -0.06017) + val pValsR = Array(0.42199, 0.51094, 0.95202) + val dispersionR = 1 + val nullDevianceR = 2.17561 + val residualDevianceR = 0.00018 + val residualDegreeOfFreedomNullR = 3 + val residualDegreeOfFreedomR = 1 + val aicR = 10.24453 val summary = model.summary val devianceResiduals = summary.residuals() @@ -1040,81 +1123,79 @@ class GeneralizedLinearRegressionSuite assert(summary.solver === "irls") } - test("glm summary: poisson family with weight") { + test("glm summary: poisson family with weight and offset") { /* - R code: + R code: - A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) - b <- c(2, 8, 3, 9) - w <- c(1, 2, 3, 4) - df <- as.data.frame(cbind(A, b)) + A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) + b <- c(2, 8, 3, 9) + w <- c(1, 2, 3, 4) + off <- c(2, 3, 1, 4) + df <- as.data.frame(cbind(A, b)) */ - val datasetWithWeight = Seq( - Instance(2.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), - Instance(8.0, 2.0, Vectors.dense(1.0, 7.0)), - Instance(3.0, 3.0, Vectors.dense(2.0, 11.0)), - Instance(9.0, 4.0, Vectors.dense(3.0, 13.0)) + val dataset = Seq( + OffsetInstance(2.0, 1.0, 2.0, Vectors.dense(0.0, 5.0).toSparse), + OffsetInstance(8.0, 2.0, 3.0, Vectors.dense(1.0, 7.0)), + OffsetInstance(3.0, 3.0, 1.0, Vectors.dense(2.0, 11.0)), + OffsetInstance(9.0, 4.0, 4.0, Vectors.dense(3.0, 13.0)) ).toDF() /* - R code: - - model <- glm(formula = "b ~ .", family="poisson", data = df, weights = w) - summary(model) - - Deviance Residuals: - 1 2 3 4 - -0.28952 0.11048 0.14839 -0.07268 - - Coefficients: - Estimate Std. Error z value Pr(>|z|) - (Intercept) 6.2999 1.6086 3.916 8.99e-05 *** - V1 3.3241 1.0184 3.264 0.00110 ** - V2 -1.0818 0.3522 -3.071 0.00213 ** - --- - Signif. codes: 0 '***' 0.001 '**' 0.01 '*' 0.05 '.' 0.1 ' ' 1 - - (Dispersion parameter for poisson family taken to be 1) - - Null deviance: 15.38066 on 3 degrees of freedom - Residual deviance: 0.12333 on 1 degrees of freedom - AIC: 41.803 - - Number of Fisher Scoring iterations: 3 + R code: - residuals(model, type="pearson") - 1 2 3 4 - -0.28043145 0.11099310 0.14963714 -0.07253611 + model <- glm(formula = "b ~ .", family = "poisson", data = df, + weights = w, offset = off) + summary(model) - residuals(model, type="working") - 1 2 3 4 - -0.17960679 0.02813593 0.05113852 -0.01201650 + Deviance Residuals: + 1 2 3 4 + -2.0480 1.2315 1.8293 -0.7107 - residuals(model, type="response") - 1 2 3 4 - -0.4378554 0.2189277 0.1459518 -0.1094638 + Coefficients: + Estimate Std. Error z value Pr(>|z|) + (Intercept) -4.5678 1.9625 -2.328 0.0199 + V1 -2.8784 1.1683 -2.464 0.0137 + V2 0.8859 0.4170 2.124 0.0336 + + (Dispersion parameter for poisson family taken to be 1) + + Null deviance: 22.5585 on 3 degrees of freedom + Residual deviance: 9.5622 on 1 degrees of freedom + AIC: 51.242 + + Number of Fisher Scoring iterations: 5 + + residuals(model, type = "pearson") + 1 2 3 4 + -1.7480418 1.3037611 2.0750099 -0.6972966 + residuals(model, type = "working") + 1 2 3 4 + -0.6891489 0.3833588 0.9710682 -0.1096590 + residuals(model, type = "response") + 1 2 3 4 + -4.433948 2.216974 1.477983 -1.108487 */ val trainer = new GeneralizedLinearRegression() .setFamily("Poisson") .setWeightCol("weight") - .setFitIntercept(true) - - val model = trainer.fit(datasetWithWeight) - - val coefficientsR = Vectors.dense(Array(3.3241, -1.0818)) - val interceptR = 6.2999 - val devianceResidualsR = Array(-0.28952, 0.11048, 0.14839, -0.07268) - val pearsonResidualsR = Array(-0.28043145, 0.11099310, 0.14963714, -0.07253611) - val workingResidualsR = Array(-0.17960679, 0.02813593, 0.05113852, -0.01201650) - val responseResidualsR = Array(-0.4378554, 0.2189277, 0.1459518, -0.1094638) - val seCoefR = Array(1.0184, 0.3522, 1.6086) - val tValsR = Array(3.264, -3.071, 3.916) - val pValsR = Array(0.00110, 0.00213, 0.00009) - val dispersionR = 1.0 - val nullDevianceR = 15.38066 - val residualDevianceR = 0.12333 + .setOffsetCol("offset") + + val model = trainer.fit(dataset) + + val coefficientsR = Vectors.dense(Array(-2.87843, 0.88589)) + val interceptR = -4.56784 + val devianceResidualsR = Array(-2.04796, 1.23149, 1.82933, -0.71066) + val pearsonResidualsR = Array(-1.74804, 1.30376, 2.07501, -0.6973) + val workingResidualsR = Array(-0.68915, 0.38336, 0.97107, -0.10966) + val responseResidualsR = Array(-4.43395, 2.21697, 1.47798, -1.10849) + val seCoefR = Array(1.16826, 0.41703, 1.96249) + val tValsR = Array(-2.46387, 2.12428, -2.32757) + val pValsR = Array(0.01374, 0.03365, 0.01993) + val dispersionR = 1 + val nullDevianceR = 22.55853 + val residualDevianceR = 9.5622 val residualDegreeOfFreedomNullR = 3 val residualDegreeOfFreedomR = 1 - val aicR = 41.803 + val aicR = 51.24218 val summary = model.summary val devianceResiduals = summary.residuals() @@ -1157,78 +1238,79 @@ class GeneralizedLinearRegressionSuite assert(summary.solver === "irls") } - test("glm summary: gamma family with weight") { + test("glm summary: gamma family with weight and offset") { /* - R code: + R code: - A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) - b <- c(2, 8, 3, 9) - w <- c(1, 2, 3, 4) - df <- as.data.frame(cbind(A, b)) + A <- matrix(c(0, 5, 1, 2, 2, 1, 3, 3), 4, 2, byrow = TRUE) + b <- c(1, 2, 1, 2) + w <- c(1, 2, 3, 4) + off <- c(0, 0.5, 1, 0) + df <- as.data.frame(cbind(A, b)) */ - val datasetWithWeight = Seq( - Instance(2.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), - Instance(8.0, 2.0, Vectors.dense(1.0, 7.0)), - Instance(3.0, 3.0, Vectors.dense(2.0, 11.0)), - Instance(9.0, 4.0, Vectors.dense(3.0, 13.0)) + val dataset = Seq( + OffsetInstance(1.0, 1.0, 0.0, Vectors.dense(0.0, 5.0)), + OffsetInstance(2.0, 2.0, 0.5, Vectors.dense(1.0, 2.0)), + OffsetInstance(1.0, 3.0, 1.0, Vectors.dense(2.0, 1.0)), + OffsetInstance(2.0, 4.0, 0.0, Vectors.dense(3.0, 3.0)) ).toDF() /* - R code: - - model <- glm(formula = "b ~ .", family="Gamma", data = df, weights = w) - summary(model) + R code: - Deviance Residuals: - 1 2 3 4 - -0.26343 0.05761 0.12818 -0.03484 + model <- glm(formula = "b ~ .", family = "Gamma", data = df, + weights = w, offset = off) + summary(model) - Coefficients: - Estimate Std. Error t value Pr(>|t|) - (Intercept) -0.81511 0.23449 -3.476 0.178 - V1 -0.72730 0.16137 -4.507 0.139 - V2 0.23894 0.05481 4.359 0.144 + Deviance Residuals: + 1 2 3 4 + -0.17095 0.19867 -0.23604 0.03241 - (Dispersion parameter for Gamma family taken to be 0.07986091) + Coefficients: + Estimate Std. Error t value Pr(>|t|) + (Intercept) -0.56474 0.23866 -2.366 0.255 + V1 0.07695 0.06931 1.110 0.467 + V2 0.28068 0.07320 3.835 0.162 - Null deviance: 2.937462 on 3 degrees of freedom - Residual deviance: 0.090358 on 1 degrees of freedom - AIC: 23.202 + (Dispersion parameter for Gamma family taken to be 0.1212174) - Number of Fisher Scoring iterations: 4 + Null deviance: 2.02568 on 3 degrees of freedom + Residual deviance: 0.12546 on 1 degrees of freedom + AIC: 0.93388 - residuals(model, type="pearson") - 1 2 3 4 - -0.24082508 0.05839241 0.13135766 -0.03463621 + Number of Fisher Scoring iterations: 4 - residuals(model, type="working") + residuals(model, type = "pearson") + 1 2 3 4 + -0.16134949 0.20807694 -0.22544551 0.03258777 + residuals(model, type = "working") 1 2 3 4 - 0.091414181 -0.005374314 -0.027196998 0.001890910 - - residuals(model, type="response") - 1 2 3 4 - -0.6344390 0.3172195 0.2114797 -0.1586097 + 0.135315831 -0.084390309 0.113219135 -0.008279688 + residuals(model, type = "response") + 1 2 3 4 + -0.1923918 0.2565224 -0.1496381 0.0320653 */ val trainer = new GeneralizedLinearRegression() .setFamily("Gamma") .setWeightCol("weight") + .setOffsetCol("offset") + + val model = trainer.fit(dataset) - val model = trainer.fit(datasetWithWeight) - - val coefficientsR = Vectors.dense(Array(-0.72730, 0.23894)) - val interceptR = -0.81511 - val devianceResidualsR = Array(-0.26343, 0.05761, 0.12818, -0.03484) - val pearsonResidualsR = Array(-0.24082508, 0.05839241, 0.13135766, -0.03463621) - val workingResidualsR = Array(0.091414181, -0.005374314, -0.027196998, 0.001890910) - val responseResidualsR = Array(-0.6344390, 0.3172195, 0.2114797, -0.1586097) - val seCoefR = Array(0.16137, 0.05481, 0.23449) - val tValsR = Array(-4.507, 4.359, -3.476) - val pValsR = Array(0.139, 0.144, 0.178) - val dispersionR = 0.07986091 - val nullDevianceR = 2.937462 - val residualDevianceR = 0.090358 + val coefficientsR = Vectors.dense(Array(0.07695, 0.28068)) + val interceptR = -0.56474 + val devianceResidualsR = Array(-0.17095, 0.19867, -0.23604, 0.03241) + val pearsonResidualsR = Array(-0.16135, 0.20808, -0.22545, 0.03259) + val workingResidualsR = Array(0.13532, -0.08439, 0.11322, -0.00828) + val responseResidualsR = Array(-0.19239, 0.25652, -0.14964, 0.03207) + val seCoefR = Array(0.06931, 0.0732, 0.23866) + val tValsR = Array(1.11031, 3.83453, -2.3663) + val pValsR = Array(0.46675, 0.16241, 0.25454) + val dispersionR = 0.12122 + val nullDevianceR = 2.02568 + val residualDevianceR = 0.12546 val residualDegreeOfFreedomNullR = 3 val residualDegreeOfFreedomR = 1 - val aicR = 23.202 + val aicR = 0.93388 val summary = model.summary val devianceResiduals = summary.residuals() @@ -1271,77 +1353,81 @@ class GeneralizedLinearRegressionSuite assert(summary.solver === "irls") } - test("glm summary: tweedie family with weight") { + test("glm summary: tweedie family with weight and offset") { /* R code: - library(statmod) df <- as.data.frame(matrix(c( - 1.0, 1.0, 0.0, 5.0, - 0.5, 2.0, 1.0, 2.0, - 1.0, 3.0, 2.0, 1.0, - 0.0, 4.0, 3.0, 3.0), 4, 4, byrow = TRUE)) + 1.0, 1.0, 1.0, 0.0, 5.0, + 0.5, 2.0, 3.0, 1.0, 2.0, + 1.0, 3.0, 2.0, 2.0, 1.0, + 0.0, 4.0, 0.0, 3.0, 3.0), 4, 5, byrow = TRUE)) + */ + val dataset = Seq( + OffsetInstance(1.0, 1.0, 1.0, Vectors.dense(0.0, 5.0)), + OffsetInstance(0.5, 2.0, 3.0, Vectors.dense(1.0, 2.0)), + OffsetInstance(1.0, 3.0, 2.0, Vectors.dense(2.0, 1.0)), + OffsetInstance(0.0, 4.0, 0.0, Vectors.dense(3.0, 3.0)) + ).toDF() + /* + R code: - model <- glm(V1 ~ -1 + V3 + V4, data = df, weights = V2, - family = tweedie(var.power = 1.6, link.power = 0)) + library(statmod) + model <- glm(V1 ~ V4 + V5, data = df, weights = V2, offset = V3, + family = tweedie(var.power = 1.6, link.power = 0.0)) summary(model) Deviance Residuals: 1 2 3 4 - 0.6210 -0.0515 1.6935 -3.2539 + 0.8917 -2.1396 1.2252 -1.7946 Coefficients: - Estimate Std. Error t value Pr(>|t|) - V3 -0.4087 0.5205 -0.785 0.515 - V4 -0.1212 0.4082 -0.297 0.794 + Estimate Std. Error t value Pr(>|t|) + (Intercept) -0.03047 3.65000 -0.008 0.995 + V4 -1.14577 1.41674 -0.809 0.567 + V5 -0.36585 0.97065 -0.377 0.771 - (Dispersion parameter for Tweedie family taken to be 3.830036) + (Dispersion parameter for Tweedie family taken to be 6.334961) - Null deviance: 20.702 on 4 degrees of freedom - Residual deviance: 13.844 on 2 degrees of freedom + Null deviance: 12.784 on 3 degrees of freedom + Residual deviance: 10.095 on 1 degrees of freedom AIC: NA - Number of Fisher Scoring iterations: 11 - - residuals(model, type="pearson") - 1 2 3 4 - 0.7383616 -0.0509458 2.2348337 -1.4552090 - residuals(model, type="working") - 1 2 3 4 - 0.83354150 -0.04103552 1.55676369 -1.00000000 - residuals(model, type="response") - 1 2 3 4 - 0.45460738 -0.02139574 0.60888055 -0.20392801 + Number of Fisher Scoring iterations: 18 + + residuals(model, type = "pearson") + 1 2 3 4 + 1.1472554 -1.4642569 1.4935199 -0.8025842 + residuals(model, type = "working") + 1 2 3 4 + 1.3624928 -0.8322375 0.9894580 -1.0000000 + residuals(model, type = "response") + 1 2 3 4 + 0.57671828 -2.48040354 0.49735052 -0.01040646 */ - val datasetWithWeight = Seq( - Instance(1.0, 1.0, Vectors.dense(0.0, 5.0)), - Instance(0.5, 2.0, Vectors.dense(1.0, 2.0)), - Instance(1.0, 3.0, Vectors.dense(2.0, 1.0)), - Instance(0.0, 4.0, Vectors.dense(3.0, 3.0)) - ).toDF() - val trainer = new GeneralizedLinearRegression() .setFamily("tweedie") .setVariancePower(1.6) .setLinkPower(0.0) .setWeightCol("weight") - .setFitIntercept(false) - - val model = trainer.fit(datasetWithWeight) - val coefficientsR = Vectors.dense(Array(-0.408746, -0.12125)) - val interceptR = 0.0 - val devianceResidualsR = Array(0.621047, -0.051515, 1.693473, -3.253946) - val pearsonResidualsR = Array(0.738362, -0.050946, 2.234834, -1.455209) - val workingResidualsR = Array(0.833541, -0.041036, 1.556764, -1.0) - val responseResidualsR = Array(0.454607, -0.021396, 0.608881, -0.203928) - val seCoefR = Array(0.520519, 0.408215) - val tValsR = Array(-0.785267, -0.297024) - val pValsR = Array(0.514549, 0.794457) - val dispersionR = 3.830036 - val nullDevianceR = 20.702 - val residualDevianceR = 13.844 - val residualDegreeOfFreedomNullR = 4 - val residualDegreeOfFreedomR = 2 + .setOffsetCol("offset") + + val model = trainer.fit(dataset) + + val coefficientsR = Vectors.dense(Array(-1.14577, -0.36585)) + val interceptR = -0.03047 + val devianceResidualsR = Array(0.89171, -2.13961, 1.2252, -1.79463) + val pearsonResidualsR = Array(1.14726, -1.46426, 1.49352, -0.80258) + val workingResidualsR = Array(1.36249, -0.83224, 0.98946, -1) + val responseResidualsR = Array(0.57672, -2.4804, 0.49735, -0.01041) + val seCoefR = Array(1.41674, 0.97065, 3.65) + val tValsR = Array(-0.80873, -0.37691, -0.00835) + val pValsR = Array(0.56707, 0.77053, 0.99468) + val dispersionR = 6.33496 + val nullDevianceR = 12.78358 + val residualDevianceR = 10.09488 + val residualDegreeOfFreedomNullR = 3 + val residualDegreeOfFreedomR = 1 val summary = model.summary From 3c2fc19d478256f8dc0ae7219fdd188030218c07 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Fri, 30 Jun 2017 20:30:26 +0800 Subject: [PATCH 002/125] [SPARK-18294][CORE] Implement commit protocol to support `mapred` package's committer ## What changes were proposed in this pull request? This PR makes the following changes: - Implement a new commit protocol `HadoopMapRedCommitProtocol` which support the old `mapred` package's committer; - Refactor SparkHadoopWriter and SparkHadoopMapReduceWriter, now they are combined together, thus we can support write through both mapred and mapreduce API by the new SparkHadoopWriter, a lot of duplicated codes are removed. After this change, it should be pretty easy for us to support the committer from both the new and the old hadoop API at high level. ## How was this patch tested? No major behavior change, passed the existing test cases. Author: Xingbo Jiang Closes #18438 from jiangxb1987/SparkHadoopWriter. --- .../io/HadoopMapRedCommitProtocol.scala | 36 ++ .../internal/io/HadoopWriteConfigUtil.scala | 79 ++++ .../io/SparkHadoopMapReduceWriter.scala | 181 -------- .../spark/internal/io/SparkHadoopWriter.scala | 393 ++++++++++++++---- .../apache/spark/rdd/PairRDDFunctions.scala | 72 +--- .../spark/rdd/PairRDDFunctionsSuite.scala | 2 +- .../OutputCommitCoordinatorSuite.scala | 35 +- 7 files changed, 461 insertions(+), 337 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/internal/io/HadoopMapRedCommitProtocol.scala create mode 100644 core/src/main/scala/org/apache/spark/internal/io/HadoopWriteConfigUtil.scala delete mode 100644 core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapRedCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapRedCommitProtocol.scala new file mode 100644 index 000000000000..ddbd624b380d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapRedCommitProtocol.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.io + +import org.apache.hadoop.mapred._ +import org.apache.hadoop.mapreduce.{TaskAttemptContext => NewTaskAttemptContext} + +/** + * An [[FileCommitProtocol]] implementation backed by an underlying Hadoop OutputCommitter + * (from the old mapred API). + * + * Unlike Hadoop's OutputCommitter, this implementation is serializable. + */ +class HadoopMapRedCommitProtocol(jobId: String, path: String) + extends HadoopMapReduceCommitProtocol(jobId, path) { + + override def setupCommitter(context: NewTaskAttemptContext): OutputCommitter = { + val config = context.getConfiguration.asInstanceOf[JobConf] + config.getOutputCommitter + } +} diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopWriteConfigUtil.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopWriteConfigUtil.scala new file mode 100644 index 000000000000..9b987e0e1bb6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopWriteConfigUtil.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.io + +import scala.reflect.ClassTag + +import org.apache.hadoop.mapreduce._ + +import org.apache.spark.SparkConf + +/** + * Interface for create output format/committer/writer used during saving an RDD using a Hadoop + * OutputFormat (both from the old mapred API and the new mapreduce API) + * + * Notes: + * 1. Implementations should throw [[IllegalArgumentException]] when wrong hadoop API is + * referenced; + * 2. Implementations must be serializable, as the instance instantiated on the driver + * will be used for tasks on executors; + * 3. Implementations should have a constructor with exactly one argument: + * (conf: SerializableConfiguration) or (conf: SerializableJobConf). + */ +abstract class HadoopWriteConfigUtil[K, V: ClassTag] extends Serializable { + + // -------------------------------------------------------------------------- + // Create JobContext/TaskAttemptContext + // -------------------------------------------------------------------------- + + def createJobContext(jobTrackerId: String, jobId: Int): JobContext + + def createTaskAttemptContext( + jobTrackerId: String, + jobId: Int, + splitId: Int, + taskAttemptId: Int): TaskAttemptContext + + // -------------------------------------------------------------------------- + // Create committer + // -------------------------------------------------------------------------- + + def createCommitter(jobId: Int): HadoopMapReduceCommitProtocol + + // -------------------------------------------------------------------------- + // Create writer + // -------------------------------------------------------------------------- + + def initWriter(taskContext: TaskAttemptContext, splitId: Int): Unit + + def write(pair: (K, V)): Unit + + def closeWriter(taskContext: TaskAttemptContext): Unit + + // -------------------------------------------------------------------------- + // Create OutputFormat + // -------------------------------------------------------------------------- + + def initOutputFormat(jobContext: JobContext): Unit + + // -------------------------------------------------------------------------- + // Verify hadoop config + // -------------------------------------------------------------------------- + + def assertConf(jobContext: JobContext, conf: SparkConf): Unit +} diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala deleted file mode 100644 index 376ff9bb19f7..000000000000 --- a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala +++ /dev/null @@ -1,181 +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.internal.io - -import java.text.SimpleDateFormat -import java.util.{Date, Locale} - -import scala.reflect.ClassTag -import scala.util.DynamicVariable - -import org.apache.hadoop.conf.{Configurable, Configuration} -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapred.{JobConf, JobID} -import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl - -import org.apache.spark.{SparkConf, SparkException, TaskContext} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.OutputMetrics -import org.apache.spark.internal.Logging -import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage -import org.apache.spark.rdd.RDD -import org.apache.spark.util.{SerializableConfiguration, Utils} - -/** - * A helper object that saves an RDD using a Hadoop OutputFormat - * (from the newer mapreduce API, not the old mapred API). - */ -private[spark] -object SparkHadoopMapReduceWriter extends Logging { - - /** - * Basic work flow of this command is: - * 1. Driver side setup, prepare the data source and hadoop configuration for the write job to - * be issued. - * 2. Issues a write job consists of one or more executor side tasks, each of which writes all - * rows within an RDD partition. - * 3. If no exception is thrown in a task, commits that task, otherwise aborts that task; If any - * exception is thrown during task commitment, also aborts that task. - * 4. If all tasks are committed, commit the job, otherwise aborts the job; If any exception is - * thrown during job commitment, also aborts the job. - */ - def write[K, V: ClassTag]( - rdd: RDD[(K, V)], - hadoopConf: Configuration): Unit = { - // Extract context and configuration from RDD. - val sparkContext = rdd.context - val stageId = rdd.id - val sparkConf = rdd.conf - val conf = new SerializableConfiguration(hadoopConf) - - // Set up a job. - val jobTrackerId = SparkHadoopWriterUtils.createJobTrackerID(new Date()) - val jobAttemptId = new TaskAttemptID(jobTrackerId, stageId, TaskType.MAP, 0, 0) - val jobContext = new TaskAttemptContextImpl(conf.value, jobAttemptId) - val format = jobContext.getOutputFormatClass - - if (SparkHadoopWriterUtils.isOutputSpecValidationEnabled(sparkConf)) { - // FileOutputFormat ignores the filesystem parameter - val jobFormat = format.newInstance - jobFormat.checkOutputSpecs(jobContext) - } - - val committer = FileCommitProtocol.instantiate( - className = classOf[HadoopMapReduceCommitProtocol].getName, - jobId = stageId.toString, - outputPath = conf.value.get("mapreduce.output.fileoutputformat.outputdir"), - isAppend = false).asInstanceOf[HadoopMapReduceCommitProtocol] - committer.setupJob(jobContext) - - // Try to write all RDD partitions as a Hadoop OutputFormat. - try { - val ret = sparkContext.runJob(rdd, (context: TaskContext, iter: Iterator[(K, V)]) => { - executeTask( - context = context, - jobTrackerId = jobTrackerId, - sparkStageId = context.stageId, - sparkPartitionId = context.partitionId, - sparkAttemptNumber = context.attemptNumber, - committer = committer, - hadoopConf = conf.value, - outputFormat = format.asInstanceOf[Class[OutputFormat[K, V]]], - iterator = iter) - }) - - committer.commitJob(jobContext, ret) - logInfo(s"Job ${jobContext.getJobID} committed.") - } catch { - case cause: Throwable => - logError(s"Aborting job ${jobContext.getJobID}.", cause) - committer.abortJob(jobContext) - throw new SparkException("Job aborted.", cause) - } - } - - /** Write an RDD partition out in a single Spark task. */ - private def executeTask[K, V: ClassTag]( - context: TaskContext, - jobTrackerId: String, - sparkStageId: Int, - sparkPartitionId: Int, - sparkAttemptNumber: Int, - committer: FileCommitProtocol, - hadoopConf: Configuration, - outputFormat: Class[_ <: OutputFormat[K, V]], - iterator: Iterator[(K, V)]): TaskCommitMessage = { - // Set up a task. - val attemptId = new TaskAttemptID(jobTrackerId, sparkStageId, TaskType.REDUCE, - sparkPartitionId, sparkAttemptNumber) - val taskContext = new TaskAttemptContextImpl(hadoopConf, attemptId) - committer.setupTask(taskContext) - - val (outputMetrics, callback) = SparkHadoopWriterUtils.initHadoopOutputMetrics(context) - - // Initiate the writer. - val taskFormat = outputFormat.newInstance() - // If OutputFormat is Configurable, we should set conf to it. - taskFormat match { - case c: Configurable => c.setConf(hadoopConf) - case _ => () - } - var writer = taskFormat.getRecordWriter(taskContext) - .asInstanceOf[RecordWriter[K, V]] - require(writer != null, "Unable to obtain RecordWriter") - var recordsWritten = 0L - - // Write all rows in RDD partition. - try { - val ret = Utils.tryWithSafeFinallyAndFailureCallbacks { - // Write rows out, release resource and commit the task. - while (iterator.hasNext) { - val pair = iterator.next() - writer.write(pair._1, pair._2) - - // Update bytes written metric every few records - SparkHadoopWriterUtils.maybeUpdateOutputMetrics(outputMetrics, callback, recordsWritten) - recordsWritten += 1 - } - if (writer != null) { - writer.close(taskContext) - writer = null - } - committer.commitTask(taskContext) - }(catchBlock = { - // If there is an error, release resource and then abort the task. - try { - if (writer != null) { - writer.close(taskContext) - writer = null - } - } finally { - committer.abortTask(taskContext) - logError(s"Task ${taskContext.getTaskAttemptID} aborted.") - } - }) - - outputMetrics.setBytesWritten(callback()) - outputMetrics.setRecordsWritten(recordsWritten) - - ret - } catch { - case t: Throwable => - throw new SparkException("Task failed while writing rows", t) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala index acc9c3857100..7d846f9354df 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala @@ -17,143 +17,374 @@ package org.apache.spark.internal.io -import java.io.IOException -import java.text.{NumberFormat, SimpleDateFormat} +import java.text.NumberFormat import java.util.{Date, Locale} +import scala.reflect.ClassTag + +import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.mapred._ -import org.apache.hadoop.mapreduce.TaskType +import org.apache.hadoop.mapreduce.{JobContext => NewJobContext, +OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, +TaskAttemptContext => NewTaskAttemptContext, TaskAttemptID => NewTaskAttemptID, TaskType} +import org.apache.hadoop.mapreduce.task.{TaskAttemptContextImpl => NewTaskAttemptContextImpl} -import org.apache.spark.SerializableWritable +import org.apache.spark.{SerializableWritable, SparkConf, SparkException, TaskContext} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging -import org.apache.spark.mapred.SparkHadoopMapRedUtil -import org.apache.spark.rdd.HadoopRDD -import org.apache.spark.util.SerializableJobConf +import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage +import org.apache.spark.rdd.{HadoopRDD, RDD} +import org.apache.spark.util.{SerializableConfiguration, SerializableJobConf, Utils} /** - * Internal helper class that saves an RDD using a Hadoop OutputFormat. - * - * Saves the RDD using a JobConf, which should contain an output key class, an output value class, - * a filename to write to, etc, exactly like in a Hadoop MapReduce job. + * A helper object that saves an RDD using a Hadoop OutputFormat. + */ +private[spark] +object SparkHadoopWriter extends Logging { + import SparkHadoopWriterUtils._ + + /** + * Basic work flow of this command is: + * 1. Driver side setup, prepare the data source and hadoop configuration for the write job to + * be issued. + * 2. Issues a write job consists of one or more executor side tasks, each of which writes all + * rows within an RDD partition. + * 3. If no exception is thrown in a task, commits that task, otherwise aborts that task; If any + * exception is thrown during task commitment, also aborts that task. + * 4. If all tasks are committed, commit the job, otherwise aborts the job; If any exception is + * thrown during job commitment, also aborts the job. + */ + def write[K, V: ClassTag]( + rdd: RDD[(K, V)], + config: HadoopWriteConfigUtil[K, V]): Unit = { + // Extract context and configuration from RDD. + val sparkContext = rdd.context + val stageId = rdd.id + + // Set up a job. + val jobTrackerId = createJobTrackerID(new Date()) + val jobContext = config.createJobContext(jobTrackerId, stageId) + config.initOutputFormat(jobContext) + + // Assert the output format/key/value class is set in JobConf. + config.assertConf(jobContext, rdd.conf) + + val committer = config.createCommitter(stageId) + committer.setupJob(jobContext) + + // Try to write all RDD partitions as a Hadoop OutputFormat. + try { + val ret = sparkContext.runJob(rdd, (context: TaskContext, iter: Iterator[(K, V)]) => { + executeTask( + context = context, + config = config, + jobTrackerId = jobTrackerId, + sparkStageId = context.stageId, + sparkPartitionId = context.partitionId, + sparkAttemptNumber = context.attemptNumber, + committer = committer, + iterator = iter) + }) + + committer.commitJob(jobContext, ret) + logInfo(s"Job ${jobContext.getJobID} committed.") + } catch { + case cause: Throwable => + logError(s"Aborting job ${jobContext.getJobID}.", cause) + committer.abortJob(jobContext) + throw new SparkException("Job aborted.", cause) + } + } + + /** Write a RDD partition out in a single Spark task. */ + private def executeTask[K, V: ClassTag]( + context: TaskContext, + config: HadoopWriteConfigUtil[K, V], + jobTrackerId: String, + sparkStageId: Int, + sparkPartitionId: Int, + sparkAttemptNumber: Int, + committer: FileCommitProtocol, + iterator: Iterator[(K, V)]): TaskCommitMessage = { + // Set up a task. + val taskContext = config.createTaskAttemptContext( + jobTrackerId, sparkStageId, sparkPartitionId, sparkAttemptNumber) + committer.setupTask(taskContext) + + val (outputMetrics, callback) = initHadoopOutputMetrics(context) + + // Initiate the writer. + config.initWriter(taskContext, sparkPartitionId) + var recordsWritten = 0L + + // Write all rows in RDD partition. + try { + val ret = Utils.tryWithSafeFinallyAndFailureCallbacks { + while (iterator.hasNext) { + val pair = iterator.next() + config.write(pair) + + // Update bytes written metric every few records + maybeUpdateOutputMetrics(outputMetrics, callback, recordsWritten) + recordsWritten += 1 + } + + config.closeWriter(taskContext) + committer.commitTask(taskContext) + }(catchBlock = { + // If there is an error, release resource and then abort the task. + try { + config.closeWriter(taskContext) + } finally { + committer.abortTask(taskContext) + logError(s"Task ${taskContext.getTaskAttemptID} aborted.") + } + }) + + outputMetrics.setBytesWritten(callback()) + outputMetrics.setRecordsWritten(recordsWritten) + + ret + } catch { + case t: Throwable => + throw new SparkException("Task failed while writing rows", t) + } + } +} + +/** + * A helper class that reads JobConf from older mapred API, creates output Format/Committer/Writer. */ private[spark] -class SparkHadoopWriter(jobConf: JobConf) extends Logging with Serializable { +class HadoopMapRedWriteConfigUtil[K, V: ClassTag](conf: SerializableJobConf) + extends HadoopWriteConfigUtil[K, V] with Logging { - private val now = new Date() - private val conf = new SerializableJobConf(jobConf) + private var outputFormat: Class[_ <: OutputFormat[K, V]] = null + private var writer: RecordWriter[K, V] = null - private var jobID = 0 - private var splitID = 0 - private var attemptID = 0 - private var jID: SerializableWritable[JobID] = null - private var taID: SerializableWritable[TaskAttemptID] = null + private def getConf: JobConf = conf.value - @transient private var writer: RecordWriter[AnyRef, AnyRef] = null - @transient private var format: OutputFormat[AnyRef, AnyRef] = null - @transient private var committer: OutputCommitter = null - @transient private var jobContext: JobContext = null - @transient private var taskContext: TaskAttemptContext = null + // -------------------------------------------------------------------------- + // Create JobContext/TaskAttemptContext + // -------------------------------------------------------------------------- - def preSetup() { - setIDs(0, 0, 0) - HadoopRDD.addLocalConfiguration("", 0, 0, 0, conf.value) + override def createJobContext(jobTrackerId: String, jobId: Int): NewJobContext = { + val jobAttemptId = new SerializableWritable(new JobID(jobTrackerId, jobId)) + new JobContextImpl(getConf, jobAttemptId.value) + } - val jCtxt = getJobContext() - getOutputCommitter().setupJob(jCtxt) + override def createTaskAttemptContext( + jobTrackerId: String, + jobId: Int, + splitId: Int, + taskAttemptId: Int): NewTaskAttemptContext = { + // Update JobConf. + HadoopRDD.addLocalConfiguration(jobTrackerId, jobId, splitId, taskAttemptId, conf.value) + // Create taskContext. + val attemptId = new TaskAttemptID(jobTrackerId, jobId, TaskType.MAP, splitId, taskAttemptId) + new TaskAttemptContextImpl(getConf, attemptId) } + // -------------------------------------------------------------------------- + // Create committer + // -------------------------------------------------------------------------- - def setup(jobid: Int, splitid: Int, attemptid: Int) { - setIDs(jobid, splitid, attemptid) - HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmmss", Locale.US).format(now), - jobid, splitID, attemptID, conf.value) + override def createCommitter(jobId: Int): HadoopMapReduceCommitProtocol = { + // Update JobConf. + HadoopRDD.addLocalConfiguration("", 0, 0, 0, getConf) + // Create commit protocol. + FileCommitProtocol.instantiate( + className = classOf[HadoopMapRedCommitProtocol].getName, + jobId = jobId.toString, + outputPath = getConf.get("mapred.output.dir"), + isAppend = false).asInstanceOf[HadoopMapReduceCommitProtocol] } - def open() { + // -------------------------------------------------------------------------- + // Create writer + // -------------------------------------------------------------------------- + + override def initWriter(taskContext: NewTaskAttemptContext, splitId: Int): Unit = { val numfmt = NumberFormat.getInstance(Locale.US) numfmt.setMinimumIntegerDigits(5) numfmt.setGroupingUsed(false) - val outputName = "part-" + numfmt.format(splitID) - val path = FileOutputFormat.getOutputPath(conf.value) + val outputName = "part-" + numfmt.format(splitId) + val path = FileOutputFormat.getOutputPath(getConf) val fs: FileSystem = { if (path != null) { - path.getFileSystem(conf.value) + path.getFileSystem(getConf) } else { - FileSystem.get(conf.value) + FileSystem.get(getConf) } } - getOutputCommitter().setupTask(getTaskContext()) - writer = getOutputFormat().getRecordWriter(fs, conf.value, outputName, Reporter.NULL) + writer = getConf.getOutputFormat + .getRecordWriter(fs, getConf, outputName, Reporter.NULL) + .asInstanceOf[RecordWriter[K, V]] + + require(writer != null, "Unable to obtain RecordWriter") } - def write(key: AnyRef, value: AnyRef) { + override def write(pair: (K, V)): Unit = { + require(writer != null, "Must call createWriter before write.") + writer.write(pair._1, pair._2) + } + + override def closeWriter(taskContext: NewTaskAttemptContext): Unit = { if (writer != null) { - writer.write(key, value) - } else { - throw new IOException("Writer is null, open() has not been called") + writer.close(Reporter.NULL) } } - def close() { - writer.close(Reporter.NULL) - } + // -------------------------------------------------------------------------- + // Create OutputFormat + // -------------------------------------------------------------------------- - def commit() { - SparkHadoopMapRedUtil.commitTask(getOutputCommitter(), getTaskContext(), jobID, splitID) + override def initOutputFormat(jobContext: NewJobContext): Unit = { + if (outputFormat == null) { + outputFormat = getConf.getOutputFormat.getClass + .asInstanceOf[Class[_ <: OutputFormat[K, V]]] + } } - def commitJob() { - val cmtr = getOutputCommitter() - cmtr.commitJob(getJobContext()) + private def getOutputFormat(): OutputFormat[K, V] = { + require(outputFormat != null, "Must call initOutputFormat first.") + + outputFormat.newInstance() } - // ********* Private Functions ********* + // -------------------------------------------------------------------------- + // Verify hadoop config + // -------------------------------------------------------------------------- + + override def assertConf(jobContext: NewJobContext, conf: SparkConf): Unit = { + val outputFormatInstance = getOutputFormat() + val keyClass = getConf.getOutputKeyClass + val valueClass = getConf.getOutputValueClass + if (outputFormatInstance == null) { + throw new SparkException("Output format class not set") + } + if (keyClass == null) { + throw new SparkException("Output key class not set") + } + if (valueClass == null) { + throw new SparkException("Output value class not set") + } + SparkHadoopUtil.get.addCredentials(getConf) + + logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + + valueClass.getSimpleName + ")") - private def getOutputFormat(): OutputFormat[AnyRef, AnyRef] = { - if (format == null) { - format = conf.value.getOutputFormat() - .asInstanceOf[OutputFormat[AnyRef, AnyRef]] + if (SparkHadoopWriterUtils.isOutputSpecValidationEnabled(conf)) { + // FileOutputFormat ignores the filesystem parameter + val ignoredFs = FileSystem.get(getConf) + getOutputFormat().checkOutputSpecs(ignoredFs, getConf) } - format + } +} + +/** + * A helper class that reads Configuration from newer mapreduce API, creates output + * Format/Committer/Writer. + */ +private[spark] +class HadoopMapReduceWriteConfigUtil[K, V: ClassTag](conf: SerializableConfiguration) + extends HadoopWriteConfigUtil[K, V] with Logging { + + private var outputFormat: Class[_ <: NewOutputFormat[K, V]] = null + private var writer: NewRecordWriter[K, V] = null + + private def getConf: Configuration = conf.value + + // -------------------------------------------------------------------------- + // Create JobContext/TaskAttemptContext + // -------------------------------------------------------------------------- + + override def createJobContext(jobTrackerId: String, jobId: Int): NewJobContext = { + val jobAttemptId = new NewTaskAttemptID(jobTrackerId, jobId, TaskType.MAP, 0, 0) + new NewTaskAttemptContextImpl(getConf, jobAttemptId) + } + + override def createTaskAttemptContext( + jobTrackerId: String, + jobId: Int, + splitId: Int, + taskAttemptId: Int): NewTaskAttemptContext = { + val attemptId = new NewTaskAttemptID( + jobTrackerId, jobId, TaskType.REDUCE, splitId, taskAttemptId) + new NewTaskAttemptContextImpl(getConf, attemptId) + } + + // -------------------------------------------------------------------------- + // Create committer + // -------------------------------------------------------------------------- + + override def createCommitter(jobId: Int): HadoopMapReduceCommitProtocol = { + FileCommitProtocol.instantiate( + className = classOf[HadoopMapReduceCommitProtocol].getName, + jobId = jobId.toString, + outputPath = getConf.get("mapreduce.output.fileoutputformat.outputdir"), + isAppend = false).asInstanceOf[HadoopMapReduceCommitProtocol] } - private def getOutputCommitter(): OutputCommitter = { - if (committer == null) { - committer = conf.value.getOutputCommitter + // -------------------------------------------------------------------------- + // Create writer + // -------------------------------------------------------------------------- + + override def initWriter(taskContext: NewTaskAttemptContext, splitId: Int): Unit = { + val taskFormat = getOutputFormat() + // If OutputFormat is Configurable, we should set conf to it. + taskFormat match { + case c: Configurable => c.setConf(getConf) + case _ => () } - committer + + writer = taskFormat.getRecordWriter(taskContext) + .asInstanceOf[NewRecordWriter[K, V]] + + require(writer != null, "Unable to obtain RecordWriter") + } + + override def write(pair: (K, V)): Unit = { + require(writer != null, "Must call createWriter before write.") + writer.write(pair._1, pair._2) } - private def getJobContext(): JobContext = { - if (jobContext == null) { - jobContext = new JobContextImpl(conf.value, jID.value) + override def closeWriter(taskContext: NewTaskAttemptContext): Unit = { + if (writer != null) { + writer.close(taskContext) + writer = null + } else { + logWarning("Writer has been closed.") } - jobContext } - private def getTaskContext(): TaskAttemptContext = { - if (taskContext == null) { - taskContext = newTaskAttemptContext(conf.value, taID.value) + // -------------------------------------------------------------------------- + // Create OutputFormat + // -------------------------------------------------------------------------- + + override def initOutputFormat(jobContext: NewJobContext): Unit = { + if (outputFormat == null) { + outputFormat = jobContext.getOutputFormatClass + .asInstanceOf[Class[_ <: NewOutputFormat[K, V]]] } - taskContext } - protected def newTaskAttemptContext( - conf: JobConf, - attemptId: TaskAttemptID): TaskAttemptContext = { - new TaskAttemptContextImpl(conf, attemptId) + private def getOutputFormat(): NewOutputFormat[K, V] = { + require(outputFormat != null, "Must call initOutputFormat first.") + + outputFormat.newInstance() } - private def setIDs(jobid: Int, splitid: Int, attemptid: Int) { - jobID = jobid - splitID = splitid - attemptID = attemptid + // -------------------------------------------------------------------------- + // Verify hadoop config + // -------------------------------------------------------------------------- - jID = new SerializableWritable[JobID](SparkHadoopWriterUtils.createJobID(now, jobid)) - taID = new SerializableWritable[TaskAttemptID]( - new TaskAttemptID(new TaskID(jID.value, TaskType.MAP, splitID), attemptID)) + override def assertConf(jobContext: NewJobContext, conf: SparkConf): Unit = { + if (SparkHadoopWriterUtils.isOutputSpecValidationEnabled(conf)) { + getOutputFormat().checkOutputSpecs(jobContext) + } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 58762cc0838c..4628fa8ba270 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -27,7 +27,6 @@ import scala.reflect.ClassTag import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} @@ -36,13 +35,11 @@ import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewO import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.annotation.Experimental -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.io.{SparkHadoopMapReduceWriter, SparkHadoopWriter, - SparkHadoopWriterUtils} +import org.apache.spark.internal.io._ import org.apache.spark.internal.Logging import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer -import org.apache.spark.util.Utils +import org.apache.spark.util.{SerializableConfiguration, SerializableJobConf, Utils} import org.apache.spark.util.collection.CompactBuffer import org.apache.spark.util.random.StratifiedSamplingUtils @@ -1082,9 +1079,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * result of using direct output committer with speculation enabled. */ def saveAsNewAPIHadoopDataset(conf: Configuration): Unit = self.withScope { - SparkHadoopMapReduceWriter.write( + val config = new HadoopMapReduceWriteConfigUtil[K, V](new SerializableConfiguration(conf)) + SparkHadoopWriter.write( rdd = self, - hadoopConf = conf) + config = config) } /** @@ -1094,62 +1092,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * MapReduce job. */ def saveAsHadoopDataset(conf: JobConf): Unit = self.withScope { - // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). - val hadoopConf = conf - val outputFormatInstance = hadoopConf.getOutputFormat - val keyClass = hadoopConf.getOutputKeyClass - val valueClass = hadoopConf.getOutputValueClass - if (outputFormatInstance == null) { - throw new SparkException("Output format class not set") - } - if (keyClass == null) { - throw new SparkException("Output key class not set") - } - if (valueClass == null) { - throw new SparkException("Output value class not set") - } - SparkHadoopUtil.get.addCredentials(hadoopConf) - - logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + - valueClass.getSimpleName + ")") - - if (SparkHadoopWriterUtils.isOutputSpecValidationEnabled(self.conf)) { - // FileOutputFormat ignores the filesystem parameter - val ignoredFs = FileSystem.get(hadoopConf) - hadoopConf.getOutputFormat.checkOutputSpecs(ignoredFs, hadoopConf) - } - - val writer = new SparkHadoopWriter(hadoopConf) - writer.preSetup() - - val writeToFile = (context: TaskContext, iter: Iterator[(K, V)]) => { - // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it - // around by taking a mod. We expect that no task will be attempted 2 billion times. - val taskAttemptId = (context.taskAttemptId % Int.MaxValue).toInt - - val (outputMetrics, callback) = SparkHadoopWriterUtils.initHadoopOutputMetrics(context) - - writer.setup(context.stageId, context.partitionId, taskAttemptId) - writer.open() - var recordsWritten = 0L - - Utils.tryWithSafeFinallyAndFailureCallbacks { - while (iter.hasNext) { - val record = iter.next() - writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) - - // Update bytes written metric every few records - SparkHadoopWriterUtils.maybeUpdateOutputMetrics(outputMetrics, callback, recordsWritten) - recordsWritten += 1 - } - }(finallyBlock = writer.close()) - writer.commit() - outputMetrics.setBytesWritten(callback()) - outputMetrics.setRecordsWritten(recordsWritten) - } - - self.context.runJob(self, writeToFile) - writer.commitJob() + val config = new HadoopMapRedWriteConfigUtil[K, V](new SerializableJobConf(conf)) + SparkHadoopWriter.write( + rdd = self, + config = config) } /** diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 02df157be377..44dd955ce869 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -561,7 +561,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { pairs.saveAsHadoopFile( "ignored", pairs.keyClass, pairs.valueClass, classOf[FakeFormatWithCallback], conf) } - assert(e.getMessage contains "failed to write") + assert(e.getCause.getMessage contains "failed to write") assert(FakeWriterWithCallback.calledBy === "write,callback,close") assert(FakeWriterWithCallback.exception != null, "exception should be captured") diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index e51e6a0d3ff6..1579b614ea5b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -18,12 +18,14 @@ package org.apache.spark.scheduler import java.io.File +import java.util.Date import java.util.concurrent.TimeoutException import scala.concurrent.duration._ import scala.language.postfixOps -import org.apache.hadoop.mapred.{JobConf, OutputCommitter, TaskAttemptContext, TaskAttemptID} +import org.apache.hadoop.mapred._ +import org.apache.hadoop.mapreduce.TaskType import org.mockito.Matchers import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock @@ -31,7 +33,7 @@ import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter import org.apache.spark._ -import org.apache.spark.internal.io.SparkHadoopWriter +import org.apache.spark.internal.io.{FileCommitProtocol, HadoopMapRedCommitProtocol, SparkHadoopWriterUtils} import org.apache.spark.rdd.{FakeOutputCommitter, RDD} import org.apache.spark.util.{ThreadUtils, Utils} @@ -214,6 +216,8 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { */ private case class OutputCommitFunctions(tempDirPath: String) { + private val jobId = new SerializableWritable(SparkHadoopWriterUtils.createJobID(new Date, 0)) + // Mock output committer that simulates a successful commit (after commit is authorized) private def successfulOutputCommitter = new FakeOutputCommitter { override def commitTask(context: TaskAttemptContext): Unit = { @@ -256,14 +260,23 @@ private case class OutputCommitFunctions(tempDirPath: String) { def jobConf = new JobConf { override def getOutputCommitter(): OutputCommitter = outputCommitter } - val sparkHadoopWriter = new SparkHadoopWriter(jobConf) { - override def newTaskAttemptContext( - conf: JobConf, - attemptId: TaskAttemptID): TaskAttemptContext = { - mock(classOf[TaskAttemptContext]) - } - } - sparkHadoopWriter.setup(ctx.stageId, ctx.partitionId, ctx.attemptNumber) - sparkHadoopWriter.commit() + + // Instantiate committer. + val committer = FileCommitProtocol.instantiate( + className = classOf[HadoopMapRedCommitProtocol].getName, + jobId = jobId.value.getId.toString, + outputPath = jobConf.get("mapred.output.dir"), + isAppend = false) + + // Create TaskAttemptContext. + // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it + // around by taking a mod. We expect that no task will be attempted 2 billion times. + val taskAttemptId = (ctx.taskAttemptId % Int.MaxValue).toInt + val attemptId = new TaskAttemptID( + new TaskID(jobId.value, TaskType.MAP, ctx.partitionId), taskAttemptId) + val taskContext = new TaskAttemptContextImpl(jobConf, attemptId) + + committer.setupTask(taskContext) + committer.commitTask(taskContext) } } From 528c9281aecc49e9bff204dd303962c705c6f237 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Fri, 30 Jun 2017 23:25:14 +0800 Subject: [PATCH 003/125] [ML] Fix scala-2.10 build failure of GeneralizedLinearRegressionSuite. ## What changes were proposed in this pull request? Fix scala-2.10 build failure of ```GeneralizedLinearRegressionSuite```. ## How was this patch tested? Build with scala-2.10. Author: Yanbo Liang Closes #18489 from yanboliang/glr. --- .../ml/regression/GeneralizedLinearRegressionSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala index cfaa57314bd6..83f1344a7bcb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala @@ -1075,7 +1075,7 @@ class GeneralizedLinearRegressionSuite val seCoefR = Array(1.23439, 0.9669, 3.56866) val tValsR = Array(0.80297, -0.65737, -0.06017) val pValsR = Array(0.42199, 0.51094, 0.95202) - val dispersionR = 1 + val dispersionR = 1.0 val nullDevianceR = 2.17561 val residualDevianceR = 0.00018 val residualDegreeOfFreedomNullR = 3 @@ -1114,7 +1114,7 @@ class GeneralizedLinearRegressionSuite assert(x._1 ~== x._2 absTol 1E-3) } summary.tValues.zip(tValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } summary.pValues.zip(pValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } - assert(summary.dispersion ~== dispersionR absTol 1E-3) + assert(summary.dispersion === dispersionR) assert(summary.nullDeviance ~== nullDevianceR absTol 1E-3) assert(summary.deviance ~== residualDevianceR absTol 1E-3) assert(summary.residualDegreeOfFreedom === residualDegreeOfFreedomR) @@ -1190,7 +1190,7 @@ class GeneralizedLinearRegressionSuite val seCoefR = Array(1.16826, 0.41703, 1.96249) val tValsR = Array(-2.46387, 2.12428, -2.32757) val pValsR = Array(0.01374, 0.03365, 0.01993) - val dispersionR = 1 + val dispersionR = 1.0 val nullDevianceR = 22.55853 val residualDevianceR = 9.5622 val residualDegreeOfFreedomNullR = 3 @@ -1229,7 +1229,7 @@ class GeneralizedLinearRegressionSuite assert(x._1 ~== x._2 absTol 1E-3) } summary.tValues.zip(tValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } summary.pValues.zip(pValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } - assert(summary.dispersion ~== dispersionR absTol 1E-3) + assert(summary.dispersion === dispersionR) assert(summary.nullDeviance ~== nullDevianceR absTol 1E-3) assert(summary.deviance ~== residualDevianceR absTol 1E-3) assert(summary.residualDegreeOfFreedom === residualDegreeOfFreedomR) From 1fe08d62f022e12f2f0161af5d8f9eac51baf1b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9B=BE=E6=9E=97=E8=A5=BF?= Date: Fri, 30 Jun 2017 19:28:43 +0100 Subject: [PATCH 004/125] [SPARK-21223] Change fileToAppInfo in FsHistoryProvider to fix concurrent issue. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit # What issue does this PR address ? Jira:https://issues.apache.org/jira/browse/SPARK-21223 fix the Thread-safety issue in FsHistoryProvider Currently, Spark HistoryServer use a HashMap named fileToAppInfo in class FsHistoryProvider to store the map of eventlog path and attemptInfo. When use ThreadPool to Replay the log files in the list and merge the list of old applications with new ones, multi thread may update fileToAppInfo at the same time, which may cause Thread-safety issues, such as falling into an infinite loop because of calling resize func of the hashtable. Author: 曾林西 Closes #18430 from zenglinxi0615/master. --- .../apache/spark/deploy/history/FsHistoryProvider.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index d05ca142b618..b2a50bd05571 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.history import java.io.{FileNotFoundException, IOException, OutputStream} import java.util.UUID -import java.util.concurrent.{Executors, ExecutorService, Future, TimeUnit} +import java.util.concurrent.{ConcurrentHashMap, Executors, ExecutorService, Future, TimeUnit} import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.mutable @@ -122,7 +122,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) @volatile private var applications: mutable.LinkedHashMap[String, FsApplicationHistoryInfo] = new mutable.LinkedHashMap() - val fileToAppInfo = new mutable.HashMap[Path, FsApplicationAttemptInfo]() + val fileToAppInfo = new ConcurrentHashMap[Path, FsApplicationAttemptInfo]() // List of application logs to be deleted by event log cleaner. private var attemptsToClean = new mutable.ListBuffer[FsApplicationAttemptInfo] @@ -321,7 +321,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // scan for modified applications, replay and merge them val logInfos: Seq[FileStatus] = statusList .filter { entry => - val prevFileSize = fileToAppInfo.get(entry.getPath()).map{_.fileSize}.getOrElse(0L) + val fileInfo = fileToAppInfo.get(entry.getPath()) + val prevFileSize = if (fileInfo != null) fileInfo.fileSize else 0L !entry.isDirectory() && // FsHistoryProvider generates a hidden file which can't be read. Accidentally // reading a garbage file is safe, but we would log an error which can be scary to @@ -475,7 +476,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) fileStatus.getLen(), appListener.appSparkVersion.getOrElse("") ) - fileToAppInfo(logPath) = attemptInfo + fileToAppInfo.put(logPath, attemptInfo) logDebug(s"Application log ${attemptInfo.logPath} loaded successfully: $attemptInfo") Some(attemptInfo) } else { From eed9c4ef859fdb75a816a3e0ce2d593b34b23444 Mon Sep 17 00:00:00 2001 From: Xiao Li Date: Fri, 30 Jun 2017 14:23:56 -0700 Subject: [PATCH 005/125] [SPARK-21129][SQL] Arguments of SQL function call should not be named expressions ### What changes were proposed in this pull request? Function argument should not be named expressions. It could cause two issues: - Misleading error message - Unexpected query results when the column name is `distinct`, which is not a reserved word in our parser. ``` spark-sql> select count(distinct c1, distinct c2) from t1; Error in query: cannot resolve '`distinct`' given input columns: [c1, c2]; line 1 pos 26; 'Project [unresolvedalias('count(c1#30, 'distinct), None)] +- SubqueryAlias t1 +- CatalogRelation `default`.`t1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [c1#30, c2#31] ``` After the fix, the error message becomes ``` spark-sql> select count(distinct c1, distinct c2) from t1; Error in query: extraneous input 'c2' expecting {')', ',', '.', '[', 'OR', 'AND', 'IN', NOT, 'BETWEEN', 'LIKE', RLIKE, 'IS', EQ, '<=>', '<>', '!=', '<', LTE, '>', GTE, '+', '-', '*', '/', '%', 'DIV', '&', '|', '||', '^'}(line 1, pos 35) == SQL == select count(distinct c1, distinct c2) from t1 -----------------------------------^^^ ``` ### How was this patch tested? Added a test case to parser suite. Author: Xiao Li Author: gatorsmile Closes #18338 from gatorsmile/parserDistinctAggFunc. --- .../spark/sql/catalyst/parser/SqlBase.g4 | 3 +- .../spark/sql/catalyst/dsl/package.scala | 1 + .../sql/catalyst/parser/AstBuilder.scala | 9 +++++- .../parser/ExpressionParserSuite.scala | 6 ++-- .../sql/catalyst/parser/PlanParserSuite.scala | 6 ++++ .../resources/sql-tests/inputs/struct.sql | 7 ++++ .../sql-tests/results/struct.sql.out | 32 ++++++++++++++++++- 7 files changed, 59 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 945603173652..7ffa15009633 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -561,6 +561,7 @@ primaryExpression | CASE whenClause+ (ELSE elseExpression=expression)? END #searchedCase | CASE value=expression whenClause+ (ELSE elseExpression=expression)? END #simpleCase | CAST '(' expression AS dataType ')' #cast + | STRUCT '(' (argument+=namedExpression (',' argument+=namedExpression)*)? ')' #struct | FIRST '(' expression (IGNORE NULLS)? ')' #first | LAST '(' expression (IGNORE NULLS)? ')' #last | POSITION '(' substr=valueExpression IN str=valueExpression ')' #position @@ -569,7 +570,7 @@ primaryExpression | qualifiedName '.' ASTERISK #star | '(' namedExpression (',' namedExpression)+ ')' #rowConstructor | '(' query ')' #subqueryExpression - | qualifiedName '(' (setQuantifier? namedExpression (',' namedExpression)*)? ')' + | qualifiedName '(' (setQuantifier? argument+=expression (',' argument+=expression)*)? ')' (OVER windowSpec)? #functionCall | value=primaryExpression '[' index=valueExpression ']' #subscript | identifier #columnReference 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 f6792569b704..7c100afcd738 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 @@ -170,6 +170,7 @@ package object dsl { case Seq() => UnresolvedStar(None) case target => UnresolvedStar(Option(target)) } + def namedStruct(e: Expression*): Expression = CreateNamedStruct(e) def callFunction[T, U]( func: T => U, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index ef79cbcaa0ce..8eac3ef2d356 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1061,6 +1061,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Cast(expression(ctx.expression), visitSparkDataType(ctx.dataType)) } + /** + * Create a [[CreateStruct]] expression. + */ + override def visitStruct(ctx: StructContext): Expression = withOrigin(ctx) { + CreateStruct(ctx.argument.asScala.map(expression)) + } + /** * Create a [[First]] expression. */ @@ -1091,7 +1098,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging // Create the function call. val name = ctx.qualifiedName.getText val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null) - val arguments = ctx.namedExpression().asScala.map(expression) match { + val arguments = ctx.argument.asScala.map(expression) match { case Seq(UnresolvedStar(None)) if name.toLowerCase(Locale.ROOT) == "count" && !isDistinct => // Transform COUNT(*) into COUNT(1). diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 4d08f016a4a1..45f9f72dccc4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -231,7 +231,7 @@ class ExpressionParserSuite extends PlanTest { assertEqual("foo(distinct a, b)", 'foo.distinctFunction('a, 'b)) assertEqual("grouping(distinct a, b)", 'grouping.distinctFunction('a, 'b)) assertEqual("`select`(all a, b)", 'select.function('a, 'b)) - assertEqual("foo(a as x, b as e)", 'foo.function('a as 'x, 'b as 'e)) + intercept("foo(a x)", "extraneous input 'x'") } test("window function expressions") { @@ -330,7 +330,9 @@ class ExpressionParserSuite extends PlanTest { assertEqual("a.b", UnresolvedAttribute("a.b")) assertEqual("`select`.b", UnresolvedAttribute("select.b")) assertEqual("(a + b).b", ('a + 'b).getField("b")) // This will fail analysis. - assertEqual("struct(a, b).b", 'struct.function('a, 'b).getField("b")) + assertEqual( + "struct(a, b).b", + namedStruct(NamePlaceholder, 'a, NamePlaceholder, 'b).getField("b")) } test("reference") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index bf15b85d5b51..5b2573fa4d60 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -223,6 +223,12 @@ class PlanParserSuite extends AnalysisTest { assertEqual(s"$sql grouping sets((a, b), (a), ())", GroupingSets(Seq(Seq('a, 'b), Seq('a), Seq()), Seq('a, 'b), table("d"), Seq('a, 'b, 'sum.function('c).as("c")))) + + val m = intercept[ParseException] { + parsePlan("SELECT a, b, count(distinct a, distinct b) as c FROM d GROUP BY a, b") + }.getMessage + assert(m.contains("extraneous input 'b'")) + } test("limit") { diff --git a/sql/core/src/test/resources/sql-tests/inputs/struct.sql b/sql/core/src/test/resources/sql-tests/inputs/struct.sql index e56344dc4de8..93a1238ab18c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/struct.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/struct.sql @@ -18,3 +18,10 @@ SELECT ID, STRUCT(ST.*,CAST(ID AS STRING) AS E) NST FROM tbl_x; -- Prepend a column to a struct SELECT ID, STRUCT(CAST(ID AS STRING) AS AA, ST.*) NST FROM tbl_x; + +-- Select a column from a struct +SELECT ID, STRUCT(ST.*).C NST FROM tbl_x; +SELECT ID, STRUCT(ST.C, ST.D).D NST FROM tbl_x; + +-- Select an alias from a struct +SELECT ID, STRUCT(ST.C as STC, ST.D as STD).STD FROM tbl_x; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/struct.sql.out b/sql/core/src/test/resources/sql-tests/results/struct.sql.out index 3e32f4619546..1da33bc736f0 100644 --- a/sql/core/src/test/resources/sql-tests/results/struct.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/struct.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 6 +-- Number of queries: 9 -- !query 0 @@ -58,3 +58,33 @@ struct> 1 {"AA":"1","C":"gamma","D":"delta"} 2 {"AA":"2","C":"epsilon","D":"eta"} 3 {"AA":"3","C":"theta","D":"iota"} + + +-- !query 6 +SELECT ID, STRUCT(ST.*).C NST FROM tbl_x +-- !query 6 schema +struct +-- !query 6 output +1 gamma +2 epsilon +3 theta + + +-- !query 7 +SELECT ID, STRUCT(ST.C, ST.D).D NST FROM tbl_x +-- !query 7 schema +struct +-- !query 7 output +1 delta +2 eta +3 iota + + +-- !query 8 +SELECT ID, STRUCT(ST.C as STC, ST.D as STD).STD FROM tbl_x +-- !query 8 schema +struct +-- !query 8 output +1 delta +2 eta +3 iota From fd1325522549937232f37215db53d6478f48644c Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 30 Jun 2017 15:11:27 -0700 Subject: [PATCH 006/125] [SPARK-21052][SQL][FOLLOW-UP] Add hash map metrics to join ## What changes were proposed in this pull request? Remove `numHashCollisions` in `BytesToBytesMap`. And change `getAverageProbesPerLookup()` to `getAverageProbesPerLookup` as suggested. ## How was this patch tested? Existing tests. Author: Liang-Chi Hsieh Closes #18480 from viirya/SPARK-21052-followup. --- .../spark/unsafe/map/BytesToBytesMap.java | 33 ------------------- .../spark/sql/execution/joins/HashJoin.scala | 2 +- .../sql/execution/joins/HashedRelation.scala | 8 ++--- 3 files changed, 5 insertions(+), 38 deletions(-) diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 4bef21b6b4e4..3b6200e74f1e 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -160,14 +160,10 @@ public final class BytesToBytesMap extends MemoryConsumer { private final boolean enablePerfMetrics; - private long timeSpentResizingNs = 0; - private long numProbes = 0; private long numKeyLookups = 0; - private long numHashCollisions = 0; - private long peakMemoryUsedBytes = 0L; private final int initialCapacity; @@ -489,10 +485,6 @@ public void safeLookup(Object keyBase, long keyOffset, int keyLength, Location l ); if (areEqual) { return; - } else { - if (enablePerfMetrics) { - numHashCollisions++; - } } } } @@ -859,16 +851,6 @@ public long getPeakMemoryUsedBytes() { return peakMemoryUsedBytes; } - /** - * Returns the total amount of time spent resizing this map (in nanoseconds). - */ - public long getTimeSpentResizingNs() { - if (!enablePerfMetrics) { - throw new IllegalStateException(); - } - return timeSpentResizingNs; - } - /** * Returns the average number of probes per key lookup. */ @@ -879,13 +861,6 @@ public double getAverageProbesPerLookup() { return (1.0 * numProbes) / numKeyLookups; } - public long getNumHashCollisions() { - if (!enablePerfMetrics) { - throw new IllegalStateException(); - } - return numHashCollisions; - } - @VisibleForTesting public int getNumDataPages() { return dataPages.size(); @@ -923,10 +898,6 @@ public void reset() { void growAndRehash() { assert(longArray != null); - long resizeStartTime = -1; - if (enablePerfMetrics) { - resizeStartTime = System.nanoTime(); - } // Store references to the old data structures to be used when we re-hash final LongArray oldLongArray = longArray; final int oldCapacity = (int) oldLongArray.size() / 2; @@ -951,9 +922,5 @@ void growAndRehash() { longArray.set(newPos * 2 + 1, hashcode); } freeArray(oldLongArray); - - if (enablePerfMetrics) { - timeSpentResizingNs += System.nanoTime() - resizeStartTime; - } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index b09edf380c2d..0396168d3f31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -215,7 +215,7 @@ trait HashJoin { // At the end of the task, we update the avg hash probe. TaskContext.get().addTaskCompletionListener(_ => - avgHashProbe.set(hashed.getAverageProbesPerLookup())) + avgHashProbe.set(hashed.getAverageProbesPerLookup)) val resultProj = createResultProjection joinedIter.map { r => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 3c702856114f..2038cb9edb67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -83,7 +83,7 @@ private[execution] sealed trait HashedRelation extends KnownSizeEstimation { /** * Returns the average number of probes per key lookup. */ - def getAverageProbesPerLookup(): Double + def getAverageProbesPerLookup: Double } private[execution] object HashedRelation { @@ -280,7 +280,7 @@ private[joins] class UnsafeHashedRelation( read(in.readInt, in.readLong, in.readBytes) } - override def getAverageProbesPerLookup(): Double = binaryMap.getAverageProbesPerLookup() + override def getAverageProbesPerLookup: Double = binaryMap.getAverageProbesPerLookup } private[joins] object UnsafeHashedRelation { @@ -776,7 +776,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap /** * Returns the average number of probes per key lookup. */ - def getAverageProbesPerLookup(): Double = numProbes.toDouble / numKeyLookups + def getAverageProbesPerLookup: Double = numProbes.toDouble / numKeyLookups } private[joins] class LongHashedRelation( @@ -829,7 +829,7 @@ private[joins] class LongHashedRelation( map = in.readObject().asInstanceOf[LongToUnsafeRowMap] } - override def getAverageProbesPerLookup(): Double = map.getAverageProbesPerLookup() + override def getAverageProbesPerLookup: Double = map.getAverageProbesPerLookup } /** From 4eb41879ce774dec1d16b2281ab1fbf41f9d418a Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 1 Jul 2017 09:25:29 +0800 Subject: [PATCH 007/125] [SPARK-17528][SQL] data should be copied properly before saving into InternalRow ## What changes were proposed in this pull request? For performance reasons, `UnsafeRow.getString`, `getStruct`, etc. return a "pointer" that points to a memory region of this unsafe row. This makes the unsafe projection a little dangerous, because all of its output rows share one instance. When we implement SQL operators, we should be careful to not cache the input rows because they may be produced by unsafe projection from child operator and thus its content may change overtime. However, when we updating values of InternalRow(e.g. in mutable projection and safe projection), we only copy UTF8String, we should also copy InternalRow, ArrayData and MapData. This PR fixes this, and also fixes the copy of vairous InternalRow, ArrayData and MapData implementations. ## How was this patch tested? new regression tests Author: Wenchen Fan Closes #18483 from cloud-fan/fix-copy. --- .../apache/spark/unsafe/types/UTF8String.java | 6 + .../spark/sql/catalyst/InternalRow.scala | 27 ++++- .../spark/sql/catalyst/expressions/Cast.scala | 2 +- .../expressions/SpecificInternalRow.scala | 12 -- .../expressions/aggregate/collect.scala | 2 +- .../expressions/aggregate/interfaces.scala | 6 + .../expressions/codegen/CodeGenerator.scala | 6 +- .../codegen/GenerateSafeProjection.scala | 2 - .../spark/sql/catalyst/expressions/rows.scala | 23 ++-- .../sql/catalyst/util/GenericArrayData.scala | 10 +- .../scala/org/apache/spark/sql/RowTest.scala | 4 - .../catalyst/expressions/MapDataSuite.scala | 57 ---------- .../codegen/GeneratedProjectionSuite.scala | 36 ++++++ .../sql/catalyst/util/ComplexDataSuite.scala | 107 ++++++++++++++++++ .../execution/vectorized/ColumnarBatch.java | 2 +- .../SortBasedAggregationIterator.scala | 15 +-- .../columnar/GenerateColumnAccessor.scala | 1 - .../execution/window/AggregateProcessor.scala | 7 +- 18 files changed, 212 insertions(+), 113 deletions(-) delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MapDataSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ComplexDataSuite.scala diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 40b9fc9534f4..9de4ca71ff6d 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -1088,6 +1088,12 @@ public UTF8String clone() { return fromBytes(getBytes()); } + public UTF8String copy() { + byte[] bytes = new byte[numBytes]; + copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes); + return fromBytes(bytes); + } + @Override public int compareTo(@Nonnull final UTF8String other) { int len = Math.min(numBytes, other.numBytes); diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala index 256f64e320be..29110640d64f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.types.{DataType, Decimal, StructType} +import org.apache.spark.unsafe.types.UTF8String /** * An abstract class for row used internally in Spark SQL, which only contains the columns as @@ -33,6 +35,10 @@ abstract class InternalRow extends SpecializedGetters with Serializable { def setNullAt(i: Int): Unit + /** + * Updates the value at column `i`. Note that after updating, the given value will be kept in this + * row, and the caller side should guarantee that this value won't be changed afterwards. + */ def update(i: Int, value: Any): Unit // default implementation (slow) @@ -58,7 +64,15 @@ abstract class InternalRow extends SpecializedGetters with Serializable { def copy(): InternalRow /** Returns true if there are any NULL values in this row. */ - def anyNull: Boolean + def anyNull: Boolean = { + val len = numFields + var i = 0 + while (i < len) { + if (isNullAt(i)) { return true } + i += 1 + } + false + } /* ---------------------- utility methods for Scala ---------------------- */ @@ -94,4 +108,15 @@ object InternalRow { /** Returns an empty [[InternalRow]]. */ val empty = apply() + + /** + * Copies the given value if it's string/struct/array/map type. + */ + def copyValue(value: Any): Any = value match { + case v: UTF8String => v.copy() + case v: InternalRow => v.copy() + case v: ArrayData => v.copy() + case v: MapData => v.copy() + case _ => value + } } 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 43df19ba009a..3862e64b9d82 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 @@ -1047,7 +1047,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String final $rowClass $result = new $rowClass(${fieldsCasts.length}); final InternalRow $tmpRow = $c; $fieldsEvalCode - $evPrim = $result.copy(); + $evPrim = $result; """ } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala index 74e0b4691d4c..75feaf670c84 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types._ /** @@ -220,17 +219,6 @@ final class SpecificInternalRow(val values: Array[MutableValue]) extends BaseGen override def isNullAt(i: Int): Boolean = values(i).isNull - override def copy(): InternalRow = { - val newValues = new Array[Any](values.length) - var i = 0 - while (i < values.length) { - newValues(i) = values(i).boxed - i += 1 - } - - new GenericInternalRow(newValues) - } - override protected def genericGet(i: Int): Any = values(i).boxed override def update(ordinal: Int, value: Any) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala index 26cd9ab66538..0d2f9889a27d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala @@ -52,7 +52,7 @@ abstract class Collect[T <: Growable[Any] with Iterable[Any]] extends TypedImper // Do not allow null values. We follow the semantics of Hive's collect_list/collect_set here. // See: org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMkCollectionEvaluator if (value != null) { - buffer += value + buffer += InternalRow.copyValue(value) } buffer } 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 fffcc7c9ef53..7af490143585 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 @@ -317,6 +317,9 @@ abstract class ImperativeAggregate extends AggregateFunction with CodegenFallbac * Updates its aggregation buffer, located in `mutableAggBuffer`, based on the given `inputRow`. * * Use `fieldNumber + mutableAggBufferOffset` to access fields of `mutableAggBuffer`. + * + * Note that, the input row may be produced by unsafe projection and it may not be safe to cache + * some fields of the input row, as the values can be changed unexpectedly. */ def update(mutableAggBuffer: InternalRow, inputRow: InternalRow): Unit @@ -326,6 +329,9 @@ abstract class ImperativeAggregate extends AggregateFunction with CodegenFallbac * * Use `fieldNumber + mutableAggBufferOffset` to access fields of `mutableAggBuffer`. * Use `fieldNumber + inputAggBufferOffset` to access fields of `inputAggBuffer`. + * + * Note that, the input row may be produced by unsafe projection and it may not be safe to cache + * some fields of the input row, as the values can be changed unexpectedly. */ def merge(mutableAggBuffer: InternalRow, inputAggBuffer: InternalRow): Unit } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 5158949b9562..b15bf2ca7c11 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -408,9 +408,11 @@ class CodegenContext { dataType match { case _ if isPrimitiveType(jt) => s"$row.set${primitiveTypeName(jt)}($ordinal, $value)" case t: DecimalType => s"$row.setDecimal($ordinal, $value, ${t.precision})" - // The UTF8String may came from UnsafeRow, otherwise clone is cheap (re-use the bytes) - case StringType => s"$row.update($ordinal, $value.clone())" case udt: UserDefinedType[_] => setColumn(row, udt.sqlType, ordinal, value) + // The UTF8String, InternalRow, ArrayData and MapData may came from UnsafeRow, we should copy + // it to avoid keeping a "pointer" to a memory region which may get updated afterwards. + case StringType | _: StructType | _: ArrayType | _: MapType => + s"$row.update($ordinal, $value.copy())" case _ => s"$row.update($ordinal, $value)" } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index f708aeff2b14..dd0419d2286d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -131,8 +131,6 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] case s: StructType => createCodeForStruct(ctx, input, s) case ArrayType(elementType, _) => createCodeForArray(ctx, input, elementType) case MapType(keyType, valueType, _) => createCodeForMap(ctx, input, keyType, valueType) - // UTF8String act as a pointer if it's inside UnsafeRow, so copy it to make it safe. - case StringType => ExprCode("", "false", s"$input.clone()") case udt: UserDefinedType[_] => convertToSafe(ctx, input, udt.sqlType) case _ => ExprCode("", "false", input) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index 751b821e1b00..65539a2f00e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -50,16 +50,6 @@ trait BaseGenericInternalRow extends InternalRow { override def getMap(ordinal: Int): MapData = getAs(ordinal) override def getStruct(ordinal: Int, numFields: Int): InternalRow = getAs(ordinal) - override def anyNull: Boolean = { - val len = numFields - var i = 0 - while (i < len) { - if (isNullAt(i)) { return true } - i += 1 - } - false - } - override def toString: String = { if (numFields == 0) { "[empty row]" @@ -79,6 +69,17 @@ trait BaseGenericInternalRow extends InternalRow { } } + override def copy(): GenericInternalRow = { + val len = numFields + val newValues = new Array[Any](len) + var i = 0 + while (i < len) { + newValues(i) = InternalRow.copyValue(genericGet(i)) + i += 1 + } + new GenericInternalRow(newValues) + } + override def equals(o: Any): Boolean = { if (!o.isInstanceOf[BaseGenericInternalRow]) { return false @@ -206,6 +207,4 @@ class GenericInternalRow(val values: Array[Any]) extends BaseGenericInternalRow override def setNullAt(i: Int): Unit = { values(i) = null} override def update(i: Int, value: Any): Unit = { values(i) = value } - - override def copy(): GenericInternalRow = this } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala index dd660c80a9c3..9e39ed9c3a77 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala @@ -49,7 +49,15 @@ class GenericArrayData(val array: Array[Any]) extends ArrayData { def this(seqOrArray: Any) = this(GenericArrayData.anyToSeq(seqOrArray)) - override def copy(): ArrayData = new GenericArrayData(array.clone()) + override def copy(): ArrayData = { + val newValues = new Array[Any](array.length) + var i = 0 + while (i < array.length) { + newValues(i) = InternalRow.copyValue(array(i)) + i += 1 + } + new GenericArrayData(newValues) + } override def numElements(): Int = array.length diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala index c9c9599e7f46..25699de33d71 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala @@ -121,10 +121,6 @@ class RowTest extends FunSpec with Matchers { externalRow should be theSameInstanceAs externalRow.copy() } - it("copy should return same ref for internal rows") { - internalRow should be theSameInstanceAs internalRow.copy() - } - it("toSeq should not expose internal state for external rows") { val modifiedValues = modifyValues(externalRow.toSeq) externalRow.toSeq should not equal modifiedValues diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MapDataSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MapDataSuite.scala deleted file mode 100644 index 25a675a90276..000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MapDataSuite.scala +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.expressions - -import scala.collection._ - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.util.ArrayBasedMapData -import org.apache.spark.sql.types.{DataType, IntegerType, MapType, StringType} -import org.apache.spark.unsafe.types.UTF8String - -class MapDataSuite extends SparkFunSuite { - - test("inequality tests") { - def u(str: String): UTF8String = UTF8String.fromString(str) - - // test data - val testMap1 = Map(u("key1") -> 1) - val testMap2 = Map(u("key1") -> 1, u("key2") -> 2) - val testMap3 = Map(u("key1") -> 1) - val testMap4 = Map(u("key1") -> 1, u("key2") -> 2) - - // ArrayBasedMapData - val testArrayMap1 = ArrayBasedMapData(testMap1.toMap) - val testArrayMap2 = ArrayBasedMapData(testMap2.toMap) - val testArrayMap3 = ArrayBasedMapData(testMap3.toMap) - val testArrayMap4 = ArrayBasedMapData(testMap4.toMap) - assert(testArrayMap1 !== testArrayMap3) - assert(testArrayMap2 !== testArrayMap4) - - // UnsafeMapData - val unsafeConverter = UnsafeProjection.create(Array[DataType](MapType(StringType, IntegerType))) - val row = new GenericInternalRow(1) - def toUnsafeMap(map: ArrayBasedMapData): UnsafeMapData = { - row.update(0, map) - val unsafeRow = unsafeConverter.apply(row) - unsafeRow.getMap(0).copy - } - assert(toUnsafeMap(testArrayMap1) !== toUnsafeMap(testArrayMap3)) - assert(toUnsafeMap(testArrayMap2) !== toUnsafeMap(testArrayMap4)) - } -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala index 58ea5b9cb52d..0cd0d8859145 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala @@ -172,4 +172,40 @@ class GeneratedProjectionSuite extends SparkFunSuite { assert(unsafe1 === unsafe3) assert(unsafe1.getStruct(1, 7) === unsafe3.getStruct(1, 7)) } + + test("MutableProjection should not cache content from the input row") { + val mutableProj = GenerateMutableProjection.generate( + Seq(BoundReference(0, new StructType().add("i", StringType), true))) + val row = new GenericInternalRow(1) + mutableProj.target(row) + + val unsafeProj = GenerateUnsafeProjection.generate( + Seq(BoundReference(0, new StructType().add("i", StringType), true))) + val unsafeRow = unsafeProj.apply(InternalRow(InternalRow(UTF8String.fromString("a")))) + + mutableProj.apply(unsafeRow) + assert(row.getStruct(0, 1).getString(0) == "a") + + // Even if the input row of the mutable projection has been changed, the target mutable row + // should keep same. + unsafeProj.apply(InternalRow(InternalRow(UTF8String.fromString("b")))) + assert(row.getStruct(0, 1).getString(0).toString == "a") + } + + test("SafeProjection should not cache content from the input row") { + val safeProj = GenerateSafeProjection.generate( + Seq(BoundReference(0, new StructType().add("i", StringType), true))) + + val unsafeProj = GenerateUnsafeProjection.generate( + Seq(BoundReference(0, new StructType().add("i", StringType), true))) + val unsafeRow = unsafeProj.apply(InternalRow(InternalRow(UTF8String.fromString("a")))) + + val row = safeProj.apply(unsafeRow) + assert(row.getStruct(0, 1).getString(0) == "a") + + // Even if the input row of the mutable projection has been changed, the target mutable row + // should keep same. + unsafeProj.apply(InternalRow(InternalRow(UTF8String.fromString("b")))) + assert(row.getStruct(0, 1).getString(0).toString == "a") + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ComplexDataSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ComplexDataSuite.scala new file mode 100644 index 000000000000..9d285916bcf4 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ComplexDataSuite.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import scala.collection._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow, SpecificInternalRow, UnsafeMapData, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.types.{DataType, IntegerType, MapType, StringType} +import org.apache.spark.unsafe.types.UTF8String + +class ComplexDataSuite extends SparkFunSuite { + def utf8(str: String): UTF8String = UTF8String.fromString(str) + + test("inequality tests for MapData") { + // test data + val testMap1 = Map(utf8("key1") -> 1) + val testMap2 = Map(utf8("key1") -> 1, utf8("key2") -> 2) + val testMap3 = Map(utf8("key1") -> 1) + val testMap4 = Map(utf8("key1") -> 1, utf8("key2") -> 2) + + // ArrayBasedMapData + val testArrayMap1 = ArrayBasedMapData(testMap1.toMap) + val testArrayMap2 = ArrayBasedMapData(testMap2.toMap) + val testArrayMap3 = ArrayBasedMapData(testMap3.toMap) + val testArrayMap4 = ArrayBasedMapData(testMap4.toMap) + assert(testArrayMap1 !== testArrayMap3) + assert(testArrayMap2 !== testArrayMap4) + + // UnsafeMapData + val unsafeConverter = UnsafeProjection.create(Array[DataType](MapType(StringType, IntegerType))) + val row = new GenericInternalRow(1) + def toUnsafeMap(map: ArrayBasedMapData): UnsafeMapData = { + row.update(0, map) + val unsafeRow = unsafeConverter.apply(row) + unsafeRow.getMap(0).copy + } + assert(toUnsafeMap(testArrayMap1) !== toUnsafeMap(testArrayMap3)) + assert(toUnsafeMap(testArrayMap2) !== toUnsafeMap(testArrayMap4)) + } + + test("GenericInternalRow.copy return a new instance that is independent from the old one") { + val project = GenerateUnsafeProjection.generate(Seq(BoundReference(0, StringType, true))) + val unsafeRow = project.apply(InternalRow(utf8("a"))) + + val genericRow = new GenericInternalRow(Array[Any](unsafeRow.getUTF8String(0))) + val copiedGenericRow = genericRow.copy() + assert(copiedGenericRow.getString(0) == "a") + project.apply(InternalRow(UTF8String.fromString("b"))) + // The copied internal row should not be changed externally. + assert(copiedGenericRow.getString(0) == "a") + } + + test("SpecificMutableRow.copy return a new instance that is independent from the old one") { + val project = GenerateUnsafeProjection.generate(Seq(BoundReference(0, StringType, true))) + val unsafeRow = project.apply(InternalRow(utf8("a"))) + + val mutableRow = new SpecificInternalRow(Seq(StringType)) + mutableRow(0) = unsafeRow.getUTF8String(0) + val copiedMutableRow = mutableRow.copy() + assert(copiedMutableRow.getString(0) == "a") + project.apply(InternalRow(UTF8String.fromString("b"))) + // The copied internal row should not be changed externally. + assert(copiedMutableRow.getString(0) == "a") + } + + test("GenericArrayData.copy return a new instance that is independent from the old one") { + val project = GenerateUnsafeProjection.generate(Seq(BoundReference(0, StringType, true))) + val unsafeRow = project.apply(InternalRow(utf8("a"))) + + val genericArray = new GenericArrayData(Array[Any](unsafeRow.getUTF8String(0))) + val copiedGenericArray = genericArray.copy() + assert(copiedGenericArray.getUTF8String(0).toString == "a") + project.apply(InternalRow(UTF8String.fromString("b"))) + // The copied array data should not be changed externally. + assert(copiedGenericArray.getUTF8String(0).toString == "a") + } + + test("copy on nested complex type") { + val project = GenerateUnsafeProjection.generate(Seq(BoundReference(0, StringType, true))) + val unsafeRow = project.apply(InternalRow(utf8("a"))) + + val arrayOfRow = new GenericArrayData(Array[Any](InternalRow(unsafeRow.getUTF8String(0)))) + val copied = arrayOfRow.copy() + assert(copied.getStruct(0, 1).getUTF8String(0).toString == "a") + project.apply(InternalRow(UTF8String.fromString("b"))) + // The copied data should not be changed externally. + assert(copied.getStruct(0, 1).getUTF8String(0).toString == "a") + } +} 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 index e23a64350cbc..34dc3af9b85c 100644 --- 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 @@ -149,7 +149,7 @@ public InternalRow copy() { } else if (dt instanceof DoubleType) { row.setDouble(i, getDouble(i)); } else if (dt instanceof StringType) { - row.update(i, getUTF8String(i)); + row.update(i, getUTF8String(i).copy()); } else if (dt instanceof BinaryType) { row.update(i, getBinary(i)); } else if (dt instanceof DecimalType) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala index bea2dce1a765..a5a444b160c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala @@ -86,17 +86,6 @@ class SortBasedAggregationIterator( // The aggregation buffer used by the sort-based aggregation. private[this] val sortBasedAggregationBuffer: InternalRow = newBuffer - // This safe projection is used to turn the input row into safe row. This is necessary - // because the input row may be produced by unsafe projection in child operator and all the - // produced rows share one byte array. However, when we update the aggregate buffer according to - // the input row, we may cache some values from input row, e.g. `Max` will keep the max value from - // input row via MutableProjection, `CollectList` will keep all values in an array via - // ImperativeAggregate framework. These values may get changed unexpectedly if the underlying - // unsafe projection update the shared byte array. By applying a safe projection to the input row, - // we can cut down the connection from input row to the shared byte array, and thus it's safe to - // cache values from input row while updating the aggregation buffer. - private[this] val safeProj: Projection = FromUnsafeProjection(valueAttributes.map(_.dataType)) - protected def initialize(): Unit = { if (inputIterator.hasNext) { initializeBuffer(sortBasedAggregationBuffer) @@ -119,7 +108,7 @@ class SortBasedAggregationIterator( // We create a variable to track if we see the next group. var findNextPartition = false // firstRowInNextGroup is the first row of this group. We first process it. - processRow(sortBasedAggregationBuffer, safeProj(firstRowInNextGroup)) + processRow(sortBasedAggregationBuffer, firstRowInNextGroup) // The search will stop when we see the next group or there is no // input row left in the iter. @@ -130,7 +119,7 @@ class SortBasedAggregationIterator( // Check if the current row belongs the current input row. if (currentGroupingKey == groupingKey) { - processRow(sortBasedAggregationBuffer, safeProj(currentRow)) + processRow(sortBasedAggregationBuffer, currentRow) } else { // We find a new group. findNextPartition = true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index d3fa0dcd2d7c..fc977f2fd553 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -56,7 +56,6 @@ class MutableUnsafeRow(val writer: UnsafeRowWriter) extends BaseGenericInternalR // all other methods inherited from GenericMutableRow are not need override protected def genericGet(ordinal: Int): Any = throw new UnsupportedOperationException override def numFields: Int = throw new UnsupportedOperationException - override def copy(): InternalRow = throw new UnsupportedOperationException } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/AggregateProcessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/AggregateProcessor.scala index 2195c6ea9594..bc141b36e63b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/AggregateProcessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/AggregateProcessor.scala @@ -145,13 +145,10 @@ private[window] final class AggregateProcessor( /** Update the buffer. */ def update(input: InternalRow): Unit = { - // TODO(hvanhovell) this sacrifices performance for correctness. We should make sure that - // MutableProjection makes copies of the complex input objects it buffer. - val copy = input.copy() - updateProjection(join(buffer, copy)) + updateProjection(join(buffer, input)) var i = 0 while (i < numImperatives) { - imperatives(i).update(buffer, copy) + imperatives(i).update(buffer, input) i += 1 } } From 61b5df567eb8ae0df4059cb0e334316fff462de9 Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Sat, 1 Jul 2017 10:01:44 +0800 Subject: [PATCH 008/125] [SPARK-21127][SQL] Update statistics after data changing commands ## What changes were proposed in this pull request? Update stats after the following data changing commands: - InsertIntoHadoopFsRelationCommand - InsertIntoHiveTable - LoadDataCommand - TruncateTableCommand - AlterTableSetLocationCommand - AlterTableDropPartitionCommand ## How was this patch tested? Added new test cases. Author: wangzhenhua Author: Zhenhua Wang Closes #18334 from wzhfy/changeStatsForOperation. --- .../apache/spark/sql/internal/SQLConf.scala | 10 + .../sql/execution/command/CommandUtils.scala | 17 +- .../spark/sql/execution/command/ddl.scala | 15 +- .../spark/sql/StatisticsCollectionSuite.scala | 77 +++++--- .../spark/sql/hive/StatisticsSuite.scala | 187 +++++++++++------- 5 files changed, 207 insertions(+), 99 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index c641e4d3a23e..25152f3e32d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -774,6 +774,14 @@ object SQLConf { .doubleConf .createWithDefault(0.05) + val AUTO_UPDATE_SIZE = + buildConf("spark.sql.statistics.autoUpdate.size") + .doc("Enables automatic update for table size once table's data is changed. Note that if " + + "the total number of files of the table is very large, this can be expensive and slow " + + "down data change commands.") + .booleanConf + .createWithDefault(false) + val CBO_ENABLED = buildConf("spark.sql.cbo.enabled") .doc("Enables CBO for estimation of plan statistics when set true.") @@ -1083,6 +1091,8 @@ class SQLConf extends Serializable with Logging { def cboEnabled: Boolean = getConf(SQLConf.CBO_ENABLED) + def autoUpdateSize: Boolean = getConf(SQLConf.AUTO_UPDATE_SIZE) + def joinReorderEnabled: Boolean = getConf(SQLConf.JOIN_REORDER_ENABLED) def joinReorderDPThreshold: Int = getConf(SQLConf.JOIN_REORDER_DP_THRESHOLD) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala index 92397607f38f..fce12cc96620 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala @@ -36,7 +36,14 @@ object CommandUtils extends Logging { def updateTableStats(sparkSession: SparkSession, table: CatalogTable): Unit = { if (table.stats.nonEmpty) { val catalog = sparkSession.sessionState.catalog - catalog.alterTableStats(table.identifier, None) + if (sparkSession.sessionState.conf.autoUpdateSize) { + val newTable = catalog.getTableMetadata(table.identifier) + val newSize = CommandUtils.calculateTotalSize(sparkSession.sessionState, newTable) + val newStats = CatalogStatistics(sizeInBytes = newSize) + catalog.alterTableStats(table.identifier, Some(newStats)) + } else { + catalog.alterTableStats(table.identifier, None) + } } } @@ -84,7 +91,9 @@ object CommandUtils extends Logging { size } - locationUri.map { p => + val startTime = System.nanoTime() + logInfo(s"Starting to calculate the total file size under path $locationUri.") + val size = locationUri.map { p => val path = new Path(p) try { val fs = path.getFileSystem(sessionState.newHadoopConf()) @@ -97,6 +106,10 @@ object CommandUtils extends Logging { 0L } }.getOrElse(0L) + val durationInMs = (System.nanoTime() - startTime) / (1000 * 1000) + logInfo(s"It took $durationInMs ms to calculate the total file size under path $locationUri.") + + size } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index ac897c1b22d7..ba7ca84f229f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -437,7 +437,20 @@ case class AlterTableAddPartitionCommand( } catalog.createPartitions(table.identifier, parts, ignoreIfExists = ifNotExists) - CommandUtils.updateTableStats(sparkSession, table) + if (table.stats.nonEmpty) { + if (sparkSession.sessionState.conf.autoUpdateSize) { + val addedSize = parts.map { part => + CommandUtils.calculateLocationSize(sparkSession.sessionState, table.identifier, + part.storage.locationUri) + }.sum + if (addedSize > 0) { + val newStats = CatalogStatistics(sizeInBytes = table.stats.get.sizeInBytes + addedSize) + catalog.alterTableStats(table.identifier, Some(newStats)) + } + } else { + catalog.alterTableStats(table.identifier, None) + } + } Seq.empty[Row] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala index b031c52dad8b..d9392de37a81 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogStatistics import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.internal.StaticSQLConf +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.sql.test.SQLTestData.ArrayData import org.apache.spark.sql.types._ @@ -178,36 +178,63 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared test("change stats after set location command") { val table = "change_stats_set_location_table" - withTable(table) { - spark.range(100).select($"id", $"id" % 5 as "value").write.saveAsTable(table) - // analyze to get initial stats - sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS id, value") - val fetched1 = checkTableStats( - table, hasSizeInBytes = true, expectedRowCounts = Some(100)) - assert(fetched1.get.sizeInBytes > 0) - assert(fetched1.get.colStats.size == 2) - - // set location command - withTempDir { newLocation => - sql(s"ALTER TABLE $table SET LOCATION '${newLocation.toURI.toString}'") - checkTableStats(table, hasSizeInBytes = false, expectedRowCounts = None) + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + spark.range(100).select($"id", $"id" % 5 as "value").write.saveAsTable(table) + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS id, value") + val fetched1 = checkTableStats( + table, hasSizeInBytes = true, expectedRowCounts = Some(100)) + assert(fetched1.get.sizeInBytes > 0) + assert(fetched1.get.colStats.size == 2) + + // set location command + val initLocation = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) + .storage.locationUri.get.toString + withTempDir { newLocation => + sql(s"ALTER TABLE $table SET LOCATION '${newLocation.toURI.toString}'") + if (autoUpdate) { + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched2.get.sizeInBytes == 0) + assert(fetched2.get.colStats.isEmpty) + + // set back to the initial location + sql(s"ALTER TABLE $table SET LOCATION '$initLocation'") + val fetched3 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched3.get.sizeInBytes == fetched1.get.sizeInBytes) + } else { + checkTableStats(table, hasSizeInBytes = false, expectedRowCounts = None) + } + } + } } } } test("change stats after insert command for datasource table") { val table = "change_stats_insert_datasource_table" - withTable(table) { - sql(s"CREATE TABLE $table (i int, j string) USING PARQUET") - // analyze to get initial stats - sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") - val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) - assert(fetched1.get.sizeInBytes == 0) - assert(fetched1.get.colStats.size == 2) - - // insert into command - sql(s"INSERT INTO TABLE $table SELECT 1, 'abc'") - checkTableStats(table, hasSizeInBytes = false, expectedRowCounts = None) + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + sql(s"CREATE TABLE $table (i int, j string) USING PARQUET") + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") + val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) + assert(fetched1.get.sizeInBytes == 0) + assert(fetched1.get.colStats.size == 2) + + // insert into command + sql(s"INSERT INTO TABLE $table SELECT 1, 'abc'") + if (autoUpdate) { + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched2.get.sizeInBytes > 0) + assert(fetched2.get.colStats.isEmpty) + } else { + checkTableStats(table, hasSizeInBytes = false, expectedRowCounts = None) + } + } + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 5fd266c2d033..c601038a2b0a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -444,88 +444,133 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto test("change stats after insert command for hive table") { val table = s"change_stats_insert_hive_table" - withTable(table) { - sql(s"CREATE TABLE $table (i int, j string)") - // analyze to get initial stats - sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") - val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) - assert(fetched1.get.sizeInBytes == 0) - assert(fetched1.get.colStats.size == 2) - - // insert into command - sql(s"INSERT INTO TABLE $table SELECT 1, 'abc'") - assert(getStatsProperties(table).isEmpty) + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + sql(s"CREATE TABLE $table (i int, j string)") + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") + val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) + assert(fetched1.get.sizeInBytes == 0) + assert(fetched1.get.colStats.size == 2) + + // insert into command + sql(s"INSERT INTO TABLE $table SELECT 1, 'abc'") + if (autoUpdate) { + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched2.get.sizeInBytes > 0) + assert(fetched2.get.colStats.isEmpty) + val statsProp = getStatsProperties(table) + assert(statsProp(STATISTICS_TOTAL_SIZE).toLong == fetched2.get.sizeInBytes) + } else { + assert(getStatsProperties(table).isEmpty) + } + } + } } } test("change stats after load data command") { val table = "change_stats_load_table" - withTable(table) { - sql(s"CREATE TABLE $table (i INT, j STRING) STORED AS PARQUET") - // analyze to get initial stats - sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") - val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) - assert(fetched1.get.sizeInBytes == 0) - assert(fetched1.get.colStats.size == 2) - - withTempDir { loadPath => - // load data command - val file = new File(loadPath + "/data") - val writer = new PrintWriter(file) - writer.write("2,xyz") - writer.close() - sql(s"LOAD DATA INPATH '${loadPath.toURI.toString}' INTO TABLE $table") - assert(getStatsProperties(table).isEmpty) + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + sql(s"CREATE TABLE $table (i INT, j STRING) STORED AS PARQUET") + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") + val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) + assert(fetched1.get.sizeInBytes == 0) + assert(fetched1.get.colStats.size == 2) + + withTempDir { loadPath => + // load data command + val file = new File(loadPath + "/data") + val writer = new PrintWriter(file) + writer.write("2,xyz") + writer.close() + sql(s"LOAD DATA INPATH '${loadPath.toURI.toString}' INTO TABLE $table") + if (autoUpdate) { + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched2.get.sizeInBytes > 0) + assert(fetched2.get.colStats.isEmpty) + val statsProp = getStatsProperties(table) + assert(statsProp(STATISTICS_TOTAL_SIZE).toLong == fetched2.get.sizeInBytes) + } else { + assert(getStatsProperties(table).isEmpty) + } + } + } } } } test("change stats after add/drop partition command") { val table = "change_stats_part_table" - withTable(table) { - sql(s"CREATE TABLE $table (i INT, j STRING) PARTITIONED BY (ds STRING, hr STRING)") - // table has two partitions initially - for (ds <- Seq("2008-04-08"); hr <- Seq("11", "12")) { - sql(s"INSERT OVERWRITE TABLE $table PARTITION (ds='$ds',hr='$hr') SELECT 1, 'a'") - } - // analyze to get initial stats - sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") - val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(2)) - assert(fetched1.get.sizeInBytes > 0) - assert(fetched1.get.colStats.size == 2) - - withTempPaths(numPaths = 2) { case Seq(dir1, dir2) => - val file1 = new File(dir1 + "/data") - val writer1 = new PrintWriter(file1) - writer1.write("1,a") - writer1.close() - - val file2 = new File(dir2 + "/data") - val writer2 = new PrintWriter(file2) - writer2.write("1,a") - writer2.close() - - // add partition command - sql( - s""" - |ALTER TABLE $table ADD - |PARTITION (ds='2008-04-09', hr='11') LOCATION '${dir1.toURI.toString}' - |PARTITION (ds='2008-04-09', hr='12') LOCATION '${dir2.toURI.toString}' - """.stripMargin) - assert(getStatsProperties(table).isEmpty) - - // generate stats again - sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") - val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(4)) - assert(fetched2.get.sizeInBytes > 0) - assert(fetched2.get.colStats.size == 2) - - // drop partition command - sql(s"ALTER TABLE $table DROP PARTITION (ds='2008-04-08'), PARTITION (hr='12')") - // only one partition left - assert(spark.sessionState.catalog.listPartitions(TableIdentifier(table)) - .map(_.spec).toSet == Set(Map("ds" -> "2008-04-09", "hr" -> "11"))) - assert(getStatsProperties(table).isEmpty) + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + sql(s"CREATE TABLE $table (i INT, j STRING) PARTITIONED BY (ds STRING, hr STRING)") + // table has two partitions initially + for (ds <- Seq("2008-04-08"); hr <- Seq("11", "12")) { + sql(s"INSERT OVERWRITE TABLE $table PARTITION (ds='$ds',hr='$hr') SELECT 1, 'a'") + } + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") + val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(2)) + assert(fetched1.get.sizeInBytes > 0) + assert(fetched1.get.colStats.size == 2) + + withTempPaths(numPaths = 2) { case Seq(dir1, dir2) => + val file1 = new File(dir1 + "/data") + val writer1 = new PrintWriter(file1) + writer1.write("1,a") + writer1.close() + + val file2 = new File(dir2 + "/data") + val writer2 = new PrintWriter(file2) + writer2.write("1,a") + writer2.close() + + // add partition command + sql( + s""" + |ALTER TABLE $table ADD + |PARTITION (ds='2008-04-09', hr='11') LOCATION '${dir1.toURI.toString}' + |PARTITION (ds='2008-04-09', hr='12') LOCATION '${dir2.toURI.toString}' + """.stripMargin) + if (autoUpdate) { + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched2.get.sizeInBytes > fetched1.get.sizeInBytes) + assert(fetched2.get.colStats.isEmpty) + val statsProp = getStatsProperties(table) + assert(statsProp(STATISTICS_TOTAL_SIZE).toLong == fetched2.get.sizeInBytes) + } else { + assert(getStatsProperties(table).isEmpty) + } + + // now the table has four partitions, generate stats again + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") + val fetched3 = checkTableStats( + table, hasSizeInBytes = true, expectedRowCounts = Some(4)) + assert(fetched3.get.sizeInBytes > 0) + assert(fetched3.get.colStats.size == 2) + + // drop partition command + sql(s"ALTER TABLE $table DROP PARTITION (ds='2008-04-08'), PARTITION (hr='12')") + assert(spark.sessionState.catalog.listPartitions(TableIdentifier(table)) + .map(_.spec).toSet == Set(Map("ds" -> "2008-04-09", "hr" -> "11"))) + // only one partition left + if (autoUpdate) { + val fetched4 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched4.get.sizeInBytes < fetched1.get.sizeInBytes) + assert(fetched4.get.colStats.isEmpty) + val statsProp = getStatsProperties(table) + assert(statsProp(STATISTICS_TOTAL_SIZE).toLong == fetched4.get.sizeInBytes) + } else { + assert(getStatsProperties(table).isEmpty) + } + } + } } } } From b1d719e7c9faeb5661a7e712b3ecefca56bf356f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 30 Jun 2017 21:10:23 -0700 Subject: [PATCH 009/125] [SPARK-21273][SQL] Propagate logical plan stats using visitor pattern and mixin ## What changes were proposed in this pull request? We currently implement statistics propagation directly in logical plan. Given we already have two different implementations, it'd make sense to actually decouple the two and add stats propagation using mixin. This would reduce the coupling between logical plan and statistics handling. This can also be a powerful pattern in the future to add additional properties (e.g. constraints). ## How was this patch tested? Should be covered by existing test cases. Author: Reynold Xin Closes #18479 from rxin/stats-trait. --- .../sql/catalyst/catalog/interface.scala | 2 +- .../plans/logical/LocalRelation.scala | 5 +- .../catalyst/plans/logical/LogicalPlan.scala | 61 +------ .../plans/logical/LogicalPlanVisitor.scala | 87 ++++++++++ .../plans/logical/basicLogicalOperators.scala | 128 +------------- .../sql/catalyst/plans/logical/hints.scala | 5 - .../BasicStatsPlanVisitor.scala | 82 +++++++++ .../statsEstimation/LogicalPlanStats.scala | 50 ++++++ .../SizeInBytesOnlyStatsPlanVisitor.scala | 163 ++++++++++++++++++ .../BasicStatsEstimationSuite.scala | 44 ----- .../StatsEstimationTestBase.scala | 2 +- .../spark/sql/execution/ExistingRDD.scala | 4 +- .../execution/columnar/InMemoryRelation.scala | 2 +- .../datasources/LogicalRelation.scala | 7 +- .../sql/execution/streaming/memory.scala | 3 +- .../PruneFileSourcePartitionsSuite.scala | 2 +- 16 files changed, 409 insertions(+), 238 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/LogicalPlanStats.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index da50b0e7e8e4..9531456434a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -438,7 +438,7 @@ case class CatalogRelation( case (attr, index) => attr.withExprId(ExprId(index + dataCols.length)) }) - override def computeStats: Statistics = { + override def computeStats(): Statistics = { // For data source tables, we will create a `LogicalRelation` and won't call this method, for // hive serde tables, we will always generate a statistics. // TODO: unify the table stats generation. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala index dc2add64b68b..1c986fbde7ad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala @@ -66,9 +66,8 @@ case class LocalRelation(output: Seq[Attribute], data: Seq[InternalRow] = Nil) } } - override def computeStats: Statistics = - Statistics(sizeInBytes = - output.map(n => BigInt(n.dataType.defaultSize)).sum * data.length) + override def computeStats(): Statistics = + Statistics(sizeInBytes = output.map(n => BigInt(n.dataType.defaultSize)).sum * data.length) def toSQL(inlineTableName: String): String = { require(data.nonEmpty) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 0d30aa76049a..8649603b1a9f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -22,11 +22,16 @@ 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.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.LogicalPlanStats import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.types.StructType -abstract class LogicalPlan extends QueryPlan[LogicalPlan] with QueryPlanConstraints with Logging { +abstract class LogicalPlan + extends QueryPlan[LogicalPlan] + with LogicalPlanStats + with QueryPlanConstraints + with Logging { private var _analyzed: Boolean = false @@ -80,40 +85,6 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with QueryPlanConstrai } } - /** A cache for the estimated statistics, such that it will only be computed once. */ - private var statsCache: Option[Statistics] = None - - /** - * Returns the estimated statistics for the current logical plan node. Under the hood, this - * method caches the return value, which is computed based on the configuration passed in the - * first time. If the configuration changes, the cache can be invalidated by calling - * [[invalidateStatsCache()]]. - */ - final def stats: Statistics = statsCache.getOrElse { - statsCache = Some(computeStats) - statsCache.get - } - - /** Invalidates the stats cache. See [[stats]] for more information. */ - final def invalidateStatsCache(): Unit = { - statsCache = None - children.foreach(_.invalidateStatsCache()) - } - - /** - * Computes [[Statistics]] for this plan. The default implementation assumes the output - * cardinality is the product of all child plan's cardinality, i.e. applies in the case - * of cartesian joins. - * - * [[LeafNode]]s must override this. - */ - protected def computeStats: Statistics = { - if (children.isEmpty) { - throw new UnsupportedOperationException(s"LeafNode $nodeName must implement statistics.") - } - Statistics(sizeInBytes = children.map(_.stats.sizeInBytes).product) - } - override def verboseStringWithSuffix: String = { super.verboseString + statsCache.map(", " + _.toString).getOrElse("") } @@ -300,6 +271,9 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with QueryPlanConstrai abstract class LeafNode extends LogicalPlan { override final def children: Seq[LogicalPlan] = Nil override def producedAttributes: AttributeSet = outputSet + + /** Leaf nodes that can survive analysis must define their own statistics. */ + def computeStats(): Statistics = throw new UnsupportedOperationException } /** @@ -331,23 +305,6 @@ abstract class UnaryNode extends LogicalPlan { } override protected def validConstraints: Set[Expression] = child.constraints - - override def computeStats: Statistics = { - // There should be some overhead in Row object, the size should not be zero when there is - // no columns, this help to prevent divide-by-zero error. - val childRowSize = child.output.map(_.dataType.defaultSize).sum + 8 - val outputRowSize = output.map(_.dataType.defaultSize).sum + 8 - // Assume there will be the same number of rows as child has. - var sizeInBytes = (child.stats.sizeInBytes * outputRowSize) / childRowSize - if (sizeInBytes == 0) { - // sizeInBytes can't be zero, or sizeInBytes of BinaryNode will also be zero - // (product of children). - sizeInBytes = 1 - } - - // Don't propagate rowCount and attributeStats, since they are not estimated here. - Statistics(sizeInBytes = sizeInBytes, hints = child.stats.hints) - } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala new file mode 100644 index 000000000000..b23045810a4f --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans.logical + +/** + * A visitor pattern for traversing a [[LogicalPlan]] tree and compute some properties. + */ +trait LogicalPlanVisitor[T] { + + def visit(p: LogicalPlan): T = p match { + case p: Aggregate => visitAggregate(p) + case p: Distinct => visitDistinct(p) + case p: Except => visitExcept(p) + case p: Expand => visitExpand(p) + case p: Filter => visitFilter(p) + case p: Generate => visitGenerate(p) + case p: GlobalLimit => visitGlobalLimit(p) + case p: Intersect => visitIntersect(p) + case p: Join => visitJoin(p) + case p: LocalLimit => visitLocalLimit(p) + case p: Pivot => visitPivot(p) + case p: Project => visitProject(p) + case p: Range => visitRange(p) + case p: Repartition => visitRepartition(p) + case p: RepartitionByExpression => visitRepartitionByExpr(p) + case p: Sample => visitSample(p) + case p: ScriptTransformation => visitScriptTransform(p) + case p: Union => visitUnion(p) + case p: ResolvedHint => visitHint(p) + case p: LogicalPlan => default(p) + } + + def default(p: LogicalPlan): T + + def visitAggregate(p: Aggregate): T + + def visitDistinct(p: Distinct): T + + def visitExcept(p: Except): T + + def visitExpand(p: Expand): T + + def visitFilter(p: Filter): T + + def visitGenerate(p: Generate): T + + def visitGlobalLimit(p: GlobalLimit): T + + def visitHint(p: ResolvedHint): T + + def visitIntersect(p: Intersect): T + + def visitJoin(p: Join): T + + def visitLocalLimit(p: LocalLimit): T + + def visitPivot(p: Pivot): T + + def visitProject(p: Project): T + + def visitRange(p: Range): T + + def visitRepartition(p: Repartition): T + + def visitRepartitionByExpr(p: RepartitionByExpression): T + + def visitSample(p: Sample): T + + def visitScriptTransform(p: ScriptTransformation): T + + def visitUnion(p: Union): T +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index e89caabf252d..0bd3166352d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -63,14 +63,6 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extend override def validConstraints: Set[Expression] = child.constraints.union(getAliasedConstraints(projectList)) - - override def computeStats: Statistics = { - if (conf.cboEnabled) { - ProjectEstimation.estimate(this).getOrElse(super.computeStats) - } else { - super.computeStats - } - } } /** @@ -137,14 +129,6 @@ case class Filter(condition: Expression, child: LogicalPlan) .filterNot(SubqueryExpression.hasCorrelatedSubquery) child.constraints.union(predicates.toSet) } - - override def computeStats: Statistics = { - if (conf.cboEnabled) { - FilterEstimation(this).estimate.getOrElse(super.computeStats) - } else { - super.computeStats - } - } } abstract class SetOperation(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { @@ -190,15 +174,6 @@ case class Intersect(left: LogicalPlan, right: LogicalPlan) extends SetOperation Some(children.flatMap(_.maxRows).min) } } - - override def computeStats: Statistics = { - val leftSize = left.stats.sizeInBytes - val rightSize = right.stats.sizeInBytes - val sizeInBytes = if (leftSize < rightSize) leftSize else rightSize - Statistics( - sizeInBytes = sizeInBytes, - hints = left.stats.hints.resetForJoin()) - } } case class Except(left: LogicalPlan, right: LogicalPlan) extends SetOperation(left, right) { @@ -207,10 +182,6 @@ case class Except(left: LogicalPlan, right: LogicalPlan) extends SetOperation(le override def output: Seq[Attribute] = left.output override protected def validConstraints: Set[Expression] = leftConstraints - - override def computeStats: Statistics = { - left.stats.copy() - } } /** Factory for constructing new `Union` nodes. */ @@ -247,11 +218,6 @@ case class Union(children: Seq[LogicalPlan]) extends LogicalPlan { children.length > 1 && childrenResolved && allChildrenCompatible } - override def computeStats: Statistics = { - val sizeInBytes = children.map(_.stats.sizeInBytes).sum - Statistics(sizeInBytes = sizeInBytes) - } - /** * Maps the constraints containing a given (original) sequence of attributes to those with a * given (reference) sequence of attributes. Given the nature of union, we expect that the @@ -355,25 +321,6 @@ case class Join( case UsingJoin(_, _) => false case _ => resolvedExceptNatural } - - override def computeStats: Statistics = { - def simpleEstimation: Statistics = joinType match { - case LeftAnti | LeftSemi => - // LeftSemi and LeftAnti won't ever be bigger than left - left.stats - case _ => - // Make sure we don't propagate isBroadcastable in other joins, because - // they could explode the size. - val stats = super.computeStats - stats.copy(hints = stats.hints.resetForJoin()) - } - - if (conf.cboEnabled) { - JoinEstimation.estimate(this).getOrElse(simpleEstimation) - } else { - simpleEstimation - } - } } /** @@ -522,14 +469,13 @@ case class Range( override def newInstance(): Range = copy(output = output.map(_.newInstance())) - override def computeStats: Statistics = { - val sizeInBytes = LongType.defaultSize * numElements - Statistics( sizeInBytes = sizeInBytes ) - } - override def simpleString: String = { s"Range ($start, $end, step=$step, splits=$numSlices)" } + + override def computeStats(): Statistics = { + Statistics(sizeInBytes = LongType.defaultSize * numElements) + } } case class Aggregate( @@ -554,25 +500,6 @@ case class Aggregate( val nonAgg = aggregateExpressions.filter(_.find(_.isInstanceOf[AggregateExpression]).isEmpty) child.constraints.union(getAliasedConstraints(nonAgg)) } - - override def computeStats: Statistics = { - def simpleEstimation: Statistics = { - if (groupingExpressions.isEmpty) { - Statistics( - sizeInBytes = EstimationUtils.getOutputSize(output, outputRowCount = 1), - rowCount = Some(1), - hints = child.stats.hints) - } else { - super.computeStats - } - } - - if (conf.cboEnabled) { - AggregateEstimation.estimate(this).getOrElse(simpleEstimation) - } else { - simpleEstimation - } - } } case class Window( @@ -671,11 +598,6 @@ case class Expand( override def references: AttributeSet = AttributeSet(projections.flatten.flatMap(_.references)) - override def computeStats: Statistics = { - val sizeInBytes = super.computeStats.sizeInBytes * projections.length - Statistics(sizeInBytes = sizeInBytes) - } - // This operator can reuse attributes (for example making them null when doing a roll up) so // the constraints of the child may no longer be valid. override protected def validConstraints: Set[Expression] = Set.empty[Expression] @@ -742,16 +664,6 @@ case class GlobalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryN case _ => None } } - override def computeStats: Statistics = { - val limit = limitExpr.eval().asInstanceOf[Int] - val childStats = child.stats - val rowCount: BigInt = childStats.rowCount.map(_.min(limit)).getOrElse(limit) - // Don't propagate column stats, because we don't know the distribution after a limit operation - Statistics( - sizeInBytes = EstimationUtils.getOutputSize(output, rowCount, childStats.attributeStats), - rowCount = Some(rowCount), - hints = childStats.hints) - } } case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { @@ -762,24 +674,6 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryNo case _ => None } } - override def computeStats: Statistics = { - val limit = limitExpr.eval().asInstanceOf[Int] - val childStats = child.stats - if (limit == 0) { - // sizeInBytes can't be zero, or sizeInBytes of BinaryNode will also be zero - // (product of children). - Statistics( - sizeInBytes = 1, - rowCount = Some(0), - hints = childStats.hints) - } else { - // The output row count of LocalLimit should be the sum of row counts from each partition. - // However, since the number of partitions is not available here, we just use statistics of - // the child. Because the distribution after a limit operation is unknown, we do not propagate - // the column stats. - childStats.copy(attributeStats = AttributeMap(Nil)) - } - } } /** @@ -828,18 +722,6 @@ case class Sample( } override def output: Seq[Attribute] = child.output - - override def computeStats: Statistics = { - val ratio = upperBound - lowerBound - val childStats = child.stats - var sizeInBytes = EstimationUtils.ceil(BigDecimal(childStats.sizeInBytes) * ratio) - if (sizeInBytes == 0) { - sizeInBytes = 1 - } - val sampledRowCount = childStats.rowCount.map(c => EstimationUtils.ceil(BigDecimal(c) * ratio)) - // Don't propagate column stats, because we don't know the distribution after a sample operation - Statistics(sizeInBytes, sampledRowCount, hints = childStats.hints) - } } /** @@ -893,7 +775,7 @@ case class RepartitionByExpression( case object OneRowRelation extends LeafNode { override def maxRows: Option[Long] = Some(1) override def output: Seq[Attribute] = Nil - override def computeStats: Statistics = Statistics(sizeInBytes = 1) + override def computeStats(): Statistics = Statistics(sizeInBytes = 1) } /** A logical plan for `dropDuplicates`. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala index 8479c702d756..29a43528124d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala @@ -42,11 +42,6 @@ case class ResolvedHint(child: LogicalPlan, hints: HintInfo = HintInfo()) override def output: Seq[Attribute] = child.output override lazy val canonicalized: LogicalPlan = child.canonicalized - - override def computeStats: Statistics = { - val stats = child.stats - stats.copy(hints = hints) - } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala new file mode 100644 index 000000000000..93908b04fb64 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans.logical.statsEstimation + +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.types.LongType + +/** + * An [[LogicalPlanVisitor]] that computes a the statistics used in a cost-based optimizer. + */ +object BasicStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { + + /** Falls back to the estimation computed by [[SizeInBytesOnlyStatsPlanVisitor]]. */ + private def fallback(p: LogicalPlan): Statistics = SizeInBytesOnlyStatsPlanVisitor.visit(p) + + override def default(p: LogicalPlan): Statistics = fallback(p) + + override def visitAggregate(p: Aggregate): Statistics = { + AggregateEstimation.estimate(p).getOrElse(fallback(p)) + } + + override def visitDistinct(p: Distinct): Statistics = fallback(p) + + override def visitExcept(p: Except): Statistics = fallback(p) + + override def visitExpand(p: Expand): Statistics = fallback(p) + + override def visitFilter(p: Filter): Statistics = { + FilterEstimation(p).estimate.getOrElse(fallback(p)) + } + + override def visitGenerate(p: Generate): Statistics = fallback(p) + + override def visitGlobalLimit(p: GlobalLimit): Statistics = fallback(p) + + override def visitHint(p: ResolvedHint): Statistics = fallback(p) + + override def visitIntersect(p: Intersect): Statistics = fallback(p) + + override def visitJoin(p: Join): Statistics = { + JoinEstimation.estimate(p).getOrElse(fallback(p)) + } + + override def visitLocalLimit(p: LocalLimit): Statistics = fallback(p) + + override def visitPivot(p: Pivot): Statistics = fallback(p) + + override def visitProject(p: Project): Statistics = { + ProjectEstimation.estimate(p).getOrElse(fallback(p)) + } + + override def visitRange(p: logical.Range): Statistics = { + val sizeInBytes = LongType.defaultSize * p.numElements + Statistics(sizeInBytes = sizeInBytes) + } + + override def visitRepartition(p: Repartition): Statistics = fallback(p) + + override def visitRepartitionByExpr(p: RepartitionByExpression): Statistics = fallback(p) + + override def visitSample(p: Sample): Statistics = fallback(p) + + override def visitScriptTransform(p: ScriptTransformation): Statistics = fallback(p) + + override def visitUnion(p: Union): Statistics = fallback(p) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/LogicalPlanStats.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/LogicalPlanStats.scala new file mode 100644 index 000000000000..8660d9355019 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/LogicalPlanStats.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.plans.logical.statsEstimation + +import org.apache.spark.sql.catalyst.plans.logical._ + +/** + * A trait to add statistics propagation to [[LogicalPlan]]. + */ +trait LogicalPlanStats { self: LogicalPlan => + + /** + * Returns the estimated statistics for the current logical plan node. Under the hood, this + * method caches the return value, which is computed based on the configuration passed in the + * first time. If the configuration changes, the cache can be invalidated by calling + * [[invalidateStatsCache()]]. + */ + def stats: Statistics = statsCache.getOrElse { + if (conf.cboEnabled) { + statsCache = Option(BasicStatsPlanVisitor.visit(self)) + } else { + statsCache = Option(SizeInBytesOnlyStatsPlanVisitor.visit(self)) + } + statsCache.get + } + + /** A cache for the estimated statistics, such that it will only be computed once. */ + protected var statsCache: Option[Statistics] = None + + /** Invalidates the stats cache. See [[stats]] for more information. */ + final def invalidateStatsCache(): Unit = { + statsCache = None + children.foreach(_.invalidateStatsCache()) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala new file mode 100644 index 000000000000..559f12072e44 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans.logical.statsEstimation + +import org.apache.spark.sql.catalyst.expressions.AttributeMap +import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.logical._ + +/** + * An [[LogicalPlanVisitor]] that computes a single dimension for plan stats: size in bytes. + */ +object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { + + /** + * A default, commonly used estimation for unary nodes. We assume the input row number is the + * same as the output row number, and compute sizes based on the column types. + */ + private def visitUnaryNode(p: UnaryNode): Statistics = { + // There should be some overhead in Row object, the size should not be zero when there is + // no columns, this help to prevent divide-by-zero error. + val childRowSize = p.child.output.map(_.dataType.defaultSize).sum + 8 + val outputRowSize = p.output.map(_.dataType.defaultSize).sum + 8 + // Assume there will be the same number of rows as child has. + var sizeInBytes = (p.child.stats.sizeInBytes * outputRowSize) / childRowSize + if (sizeInBytes == 0) { + // sizeInBytes can't be zero, or sizeInBytes of BinaryNode will also be zero + // (product of children). + sizeInBytes = 1 + } + + // Don't propagate rowCount and attributeStats, since they are not estimated here. + Statistics(sizeInBytes = sizeInBytes, hints = p.child.stats.hints) + } + + /** + * For leaf nodes, use its computeStats. For other nodes, we assume the size in bytes is the + * sum of all of the children's. + */ + override def default(p: LogicalPlan): Statistics = p match { + case p: LeafNode => p.computeStats() + case _: LogicalPlan => Statistics(sizeInBytes = p.children.map(_.stats.sizeInBytes).product) + } + + override def visitAggregate(p: Aggregate): Statistics = { + if (p.groupingExpressions.isEmpty) { + Statistics( + sizeInBytes = EstimationUtils.getOutputSize(p.output, outputRowCount = 1), + rowCount = Some(1), + hints = p.child.stats.hints) + } else { + visitUnaryNode(p) + } + } + + override def visitDistinct(p: Distinct): Statistics = default(p) + + override def visitExcept(p: Except): Statistics = p.left.stats.copy() + + override def visitExpand(p: Expand): Statistics = { + val sizeInBytes = visitUnaryNode(p).sizeInBytes * p.projections.length + Statistics(sizeInBytes = sizeInBytes) + } + + override def visitFilter(p: Filter): Statistics = visitUnaryNode(p) + + override def visitGenerate(p: Generate): Statistics = default(p) + + override def visitGlobalLimit(p: GlobalLimit): Statistics = { + val limit = p.limitExpr.eval().asInstanceOf[Int] + val childStats = p.child.stats + val rowCount: BigInt = childStats.rowCount.map(_.min(limit)).getOrElse(limit) + // Don't propagate column stats, because we don't know the distribution after limit + Statistics( + sizeInBytes = EstimationUtils.getOutputSize(p.output, rowCount, childStats.attributeStats), + rowCount = Some(rowCount), + hints = childStats.hints) + } + + override def visitHint(p: ResolvedHint): Statistics = p.child.stats.copy(hints = p.hints) + + override def visitIntersect(p: Intersect): Statistics = { + val leftSize = p.left.stats.sizeInBytes + val rightSize = p.right.stats.sizeInBytes + val sizeInBytes = if (leftSize < rightSize) leftSize else rightSize + Statistics( + sizeInBytes = sizeInBytes, + hints = p.left.stats.hints.resetForJoin()) + } + + override def visitJoin(p: Join): Statistics = { + p.joinType match { + case LeftAnti | LeftSemi => + // LeftSemi and LeftAnti won't ever be bigger than left + p.left.stats + case _ => + // Make sure we don't propagate isBroadcastable in other joins, because + // they could explode the size. + val stats = default(p) + stats.copy(hints = stats.hints.resetForJoin()) + } + } + + override def visitLocalLimit(p: LocalLimit): Statistics = { + val limit = p.limitExpr.eval().asInstanceOf[Int] + val childStats = p.child.stats + if (limit == 0) { + // sizeInBytes can't be zero, or sizeInBytes of BinaryNode will also be zero + // (product of children). + Statistics(sizeInBytes = 1, rowCount = Some(0), hints = childStats.hints) + } else { + // The output row count of LocalLimit should be the sum of row counts from each partition. + // However, since the number of partitions is not available here, we just use statistics of + // the child. Because the distribution after a limit operation is unknown, we do not propagate + // the column stats. + childStats.copy(attributeStats = AttributeMap(Nil)) + } + } + + override def visitPivot(p: Pivot): Statistics = default(p) + + override def visitProject(p: Project): Statistics = visitUnaryNode(p) + + override def visitRange(p: logical.Range): Statistics = { + p.computeStats() + } + + override def visitRepartition(p: Repartition): Statistics = default(p) + + override def visitRepartitionByExpr(p: RepartitionByExpression): Statistics = default(p) + + override def visitSample(p: Sample): Statistics = { + val ratio = p.upperBound - p.lowerBound + var sizeInBytes = EstimationUtils.ceil(BigDecimal(p.child.stats.sizeInBytes) * ratio) + if (sizeInBytes == 0) { + sizeInBytes = 1 + } + val sampleRows = p.child.stats.rowCount.map(c => EstimationUtils.ceil(BigDecimal(c) * ratio)) + // Don't propagate column stats, because we don't know the distribution after a sample operation + Statistics(sizeInBytes, sampleRows, hints = p.child.stats.hints) + } + + override def visitScriptTransform(p: ScriptTransformation): Statistics = default(p) + + override def visitUnion(p: Union): Statistics = { + Statistics(sizeInBytes = p.children.map(_.stats.sizeInBytes).sum) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala index 912c5fed6345..31a8cbdee977 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala @@ -77,37 +77,6 @@ class BasicStatsEstimationSuite extends StatsEstimationTestBase { checkStats(globalLimit, stats) } - test("sample estimation") { - val sample = Sample(0.0, 0.5, withReplacement = false, (math.random * 1000).toLong, plan) - checkStats(sample, Statistics(sizeInBytes = 60, rowCount = Some(5))) - - // Child doesn't have rowCount in stats - val childStats = Statistics(sizeInBytes = 120) - val childPlan = DummyLogicalPlan(childStats, childStats) - val sample2 = - Sample(0.0, 0.11, withReplacement = false, (math.random * 1000).toLong, childPlan) - checkStats(sample2, Statistics(sizeInBytes = 14)) - } - - test("estimate statistics when the conf changes") { - val expectedDefaultStats = - Statistics( - sizeInBytes = 40, - rowCount = Some(10), - attributeStats = AttributeMap(Seq( - AttributeReference("c1", IntegerType)() -> ColumnStat(10, Some(1), Some(10), 0, 4, 4)))) - val expectedCboStats = - Statistics( - sizeInBytes = 4, - rowCount = Some(1), - attributeStats = AttributeMap(Seq( - AttributeReference("c1", IntegerType)() -> ColumnStat(1, Some(5), Some(5), 0, 4, 4)))) - - val plan = DummyLogicalPlan(defaultStats = expectedDefaultStats, cboStats = expectedCboStats) - checkStats( - plan, expectedStatsCboOn = expectedCboStats, expectedStatsCboOff = expectedDefaultStats) - } - /** Check estimated stats when cbo is turned on/off. */ private def checkStats( plan: LogicalPlan, @@ -132,16 +101,3 @@ class BasicStatsEstimationSuite extends StatsEstimationTestBase { private def checkStats(plan: LogicalPlan, expectedStats: Statistics): Unit = checkStats(plan, expectedStats, expectedStats) } - -/** - * This class is used for unit-testing the cbo switch, it mimics a logical plan which computes - * a simple statistics or a cbo estimated statistics based on the conf. - */ -private case class DummyLogicalPlan( - defaultStats: Statistics, - cboStats: Statistics) extends LogicalPlan { - override def output: Seq[Attribute] = Nil - override def children: Seq[LogicalPlan] = Nil - override def computeStats: Statistics = - if (conf.cboEnabled) cboStats else defaultStats -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala index eaa33e44a6a5..31dea2e3e7f1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala @@ -65,7 +65,7 @@ case class StatsTestPlan( attributeStats: AttributeMap[ColumnStat], size: Option[BigInt] = None) extends LeafNode { override def output: Seq[Attribute] = outputList - override def computeStats: Statistics = Statistics( + override def computeStats(): Statistics = Statistics( // If sizeInBytes is useless in testing, we just use a fake value sizeInBytes = size.getOrElse(Int.MaxValue), rowCount = Some(rowCount), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 66f66a289a06..dcb918eeb9d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -88,7 +88,7 @@ case class ExternalRDD[T]( override protected def stringArgs: Iterator[Any] = Iterator(output) - @transient override def computeStats: Statistics = Statistics( + override def computeStats(): Statistics = Statistics( // TODO: Instead of returning a default value here, find a way to return a meaningful size // estimate for RDDs. See PR 1238 for more discussions. sizeInBytes = BigInt(session.sessionState.conf.defaultSizeInBytes) @@ -156,7 +156,7 @@ case class LogicalRDD( override protected def stringArgs: Iterator[Any] = Iterator(output) - @transient override def computeStats: Statistics = Statistics( + override def computeStats(): Statistics = Statistics( // TODO: Instead of returning a default value here, find a way to return a meaningful size // estimate for RDDs. See PR 1238 for more discussions. sizeInBytes = BigInt(session.sessionState.conf.defaultSizeInBytes) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 2972132336de..39cf8fcac511 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -69,7 +69,7 @@ case class InMemoryRelation( @transient val partitionStatistics = new PartitionStatistics(output) - override def computeStats: Statistics = { + override def computeStats(): Statistics = { if (batchStats.value == 0L) { // Underlying columnar RDD hasn't been materialized, no useful statistics information // available, return the default statistics. 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 6ba190b9e5dc..699f1bad9c4e 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 @@ -48,9 +48,10 @@ case class LogicalRelation( output = output.map(QueryPlan.normalizeExprId(_, output)), catalogTable = None) - @transient override def computeStats: Statistics = { - catalogTable.flatMap(_.stats.map(_.toPlanStats(output))).getOrElse( - Statistics(sizeInBytes = relation.sizeInBytes)) + override def computeStats(): Statistics = { + catalogTable + .flatMap(_.stats.map(_.toPlanStats(output))) + .getOrElse(Statistics(sizeInBytes = relation.sizeInBytes)) } /** Used to lookup original attribute capitalization */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index 4979873ee3c7..587ae2bfb63f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -230,6 +230,5 @@ case class MemoryPlan(sink: MemorySink, output: Seq[Attribute]) extends LeafNode private val sizePerRow = sink.schema.toAttributes.map(_.dataType.defaultSize).sum - override def computeStats: Statistics = - Statistics(sizePerRow * sink.allData.size) + override def computeStats(): Statistics = Statistics(sizePerRow * sink.allData.size) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala index 3a724aa14f2a..94384185d190 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala @@ -86,7 +86,7 @@ class PruneFileSourcePartitionsSuite extends QueryTest with SQLTestUtils with Te case relation: LogicalRelation => relation } assert(relations.size === 1, s"Size wrong for:\n ${df.queryExecution}") - val size2 = relations(0).computeStats.sizeInBytes + val size2 = relations(0).stats.sizeInBytes assert(size2 == relations(0).catalogTable.get.stats.get.sizeInBytes) assert(size2 < tableStats.get.sizeInBytes) } From 37ef32e515ea071afe63b56ba0d4299bb76e8a75 Mon Sep 17 00:00:00 2001 From: actuaryzhang Date: Sat, 1 Jul 2017 14:57:57 +0800 Subject: [PATCH 010/125] [SPARK-21275][ML] Update GLM test to use supportedFamilyNames ## What changes were proposed in this pull request? Update GLM test to use supportedFamilyNames as suggested here: https://github.com/apache/spark/pull/16699#discussion-diff-100574976R855 Author: actuaryzhang Closes #18495 from actuaryzhang/mlGlmTest2. --- .../GeneralizedLinearRegressionSuite.scala | 33 +++++++++---------- 1 file changed, 16 insertions(+), 17 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala index 83f1344a7bcb..a47bd17f47bb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala @@ -749,15 +749,15 @@ class GeneralizedLinearRegressionSuite library(statmod) y <- c(1.0, 0.5, 0.7, 0.3) w <- c(1, 2, 3, 4) - for (fam in list(gaussian(), poisson(), binomial(), Gamma(), tweedie(1.6))) { + for (fam in list(binomial(), Gamma(), gaussian(), poisson(), tweedie(1.6))) { model1 <- glm(y ~ 1, family = fam) model2 <- glm(y ~ 1, family = fam, weights = w) print(as.vector(c(coef(model1), coef(model2)))) } - [1] 0.625 0.530 - [1] -0.4700036 -0.6348783 [1] 0.5108256 0.1201443 [1] 1.600000 1.886792 + [1] 0.625 0.530 + [1] -0.4700036 -0.6348783 [1] 1.325782 1.463641 */ @@ -768,13 +768,13 @@ class GeneralizedLinearRegressionSuite Instance(0.3, 4.0, Vectors.zeros(0)) ).toDF() - val expected = Seq(0.625, 0.530, -0.4700036, -0.6348783, 0.5108256, 0.1201443, - 1.600000, 1.886792, 1.325782, 1.463641) + val expected = Seq(0.5108256, 0.1201443, 1.600000, 1.886792, 0.625, 0.530, + -0.4700036, -0.6348783, 1.325782, 1.463641) import GeneralizedLinearRegression._ var idx = 0 - for (family <- Seq("gaussian", "poisson", "binomial", "gamma", "tweedie")) { + for (family <- GeneralizedLinearRegression.supportedFamilyNames.sortWith(_ < _)) { for (useWeight <- Seq(false, true)) { val trainer = new GeneralizedLinearRegression().setFamily(family) if (useWeight) trainer.setWeightCol("weight") @@ -807,7 +807,7 @@ class GeneralizedLinearRegressionSuite 0.5, 2.1, 0.5, 1.0, 2.0, 0.9, 0.4, 1.0, 2.0, 1.0, 0.7, 0.7, 0.0, 3.0, 3.0), 4, 5, byrow = TRUE)) - families <- list(gaussian, binomial, poisson, Gamma, tweedie(1.5)) + families <- list(binomial, Gamma, gaussian, poisson, tweedie(1.5)) f1 <- V1 ~ -1 + V4 + V5 f2 <- V1 ~ V4 + V5 for (f in c(f1, f2)) { @@ -816,15 +816,15 @@ class GeneralizedLinearRegressionSuite print(as.vector(coef(model))) } } - [1] 0.5169222 -0.3344444 [1] 0.9419107 -0.6864404 - [1] 0.1812436 -0.6568422 [1] -0.2869094 0.7857710 + [1] 0.5169222 -0.3344444 + [1] 0.1812436 -0.6568422 [1] 0.1055254 0.2979113 - [1] -0.05990345 0.53188982 -0.32118415 [1] -0.2147117 0.9911750 -0.6356096 - [1] -1.5616130 0.6646470 -0.3192581 [1] 0.3390397 -0.3406099 0.6870259 + [1] -0.05990345 0.53188982 -0.32118415 + [1] -1.5616130 0.6646470 -0.3192581 [1] 0.3665034 0.1039416 0.1484616 */ val dataset = Seq( @@ -835,23 +835,22 @@ class GeneralizedLinearRegressionSuite ).toDF() val expected = Seq( - Vectors.dense(0, 0.5169222, -0.3344444), Vectors.dense(0, 0.9419107, -0.6864404), - Vectors.dense(0, 0.1812436, -0.6568422), Vectors.dense(0, -0.2869094, 0.785771), + Vectors.dense(0, 0.5169222, -0.3344444), + Vectors.dense(0, 0.1812436, -0.6568422), Vectors.dense(0, 0.1055254, 0.2979113), - Vectors.dense(-0.05990345, 0.53188982, -0.32118415), Vectors.dense(-0.2147117, 0.991175, -0.6356096), - Vectors.dense(-1.561613, 0.664647, -0.3192581), Vectors.dense(0.3390397, -0.3406099, 0.6870259), + Vectors.dense(-0.05990345, 0.53188982, -0.32118415), + Vectors.dense(-1.561613, 0.664647, -0.3192581), Vectors.dense(0.3665034, 0.1039416, 0.1484616)) import GeneralizedLinearRegression._ var idx = 0 - for (fitIntercept <- Seq(false, true)) { - for (family <- Seq("gaussian", "binomial", "poisson", "gamma", "tweedie")) { + for (family <- GeneralizedLinearRegression.supportedFamilyNames.sortWith(_ < _)) { val trainer = new GeneralizedLinearRegression().setFamily(family) .setFitIntercept(fitIntercept).setOffsetCol("offset") .setWeightCol("weight").setLinkPredictionCol("linkPrediction") From e0b047eafed92eadf6842a9df964438095e12d41 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Sat, 1 Jul 2017 15:37:41 +0800 Subject: [PATCH 011/125] [SPARK-18518][ML] HasSolver supports override ## What changes were proposed in this pull request? 1, make param support non-final with `finalFields` option 2, generate `HasSolver` with `finalFields = false` 3, override `solver` in LiR, GLR, and make MLPC inherit `HasSolver` ## How was this patch tested? existing tests Author: Ruifeng Zheng Author: Zheng RuiFeng Closes #16028 from zhengruifeng/param_non_final. --- .../MultilayerPerceptronClassifier.scala | 19 ++++---- .../ml/param/shared/SharedParamsCodeGen.scala | 11 +++-- .../spark/ml/param/shared/sharedParams.scala | 8 ++-- .../GeneralizedLinearRegression.scala | 21 ++++++++- .../ml/regression/LinearRegression.scala | 46 +++++++++++++++---- python/pyspark/ml/classification.py | 18 +------- python/pyspark/ml/regression.py | 5 ++ 7 files changed, 82 insertions(+), 46 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index ec39f964e213..ceba11edc93b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -27,13 +27,16 @@ import org.apache.spark.ml.ann.{FeedForwardTopology, FeedForwardTrainer} import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasMaxIter, HasSeed, HasStepSize, HasTol} +import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.sql.Dataset /** Params for Multilayer Perceptron. */ private[classification] trait MultilayerPerceptronParams extends PredictorParams - with HasSeed with HasMaxIter with HasTol with HasStepSize { + with HasSeed with HasMaxIter with HasTol with HasStepSize with HasSolver { + + import MultilayerPerceptronClassifier._ + /** * Layer sizes including input size and output size. * @@ -78,14 +81,10 @@ private[classification] trait MultilayerPerceptronParams extends PredictorParams * @group expertParam */ @Since("2.0.0") - final val solver: Param[String] = new Param[String](this, "solver", + final override val solver: Param[String] = new Param[String](this, "solver", "The solver algorithm for optimization. Supported options: " + - s"${MultilayerPerceptronClassifier.supportedSolvers.mkString(", ")}. (Default l-bfgs)", - ParamValidators.inArray[String](MultilayerPerceptronClassifier.supportedSolvers)) - - /** @group expertGetParam */ - @Since("2.0.0") - final def getSolver: String = $(solver) + s"${supportedSolvers.mkString(", ")}. (Default l-bfgs)", + ParamValidators.inArray[String](supportedSolvers)) /** * The initial weights of the model. @@ -101,7 +100,7 @@ private[classification] trait MultilayerPerceptronParams extends PredictorParams final def getInitialWeights: Vector = $(initialWeights) setDefault(maxIter -> 100, tol -> 1e-6, blockSize -> 128, - solver -> MultilayerPerceptronClassifier.LBFGS, stepSize -> 0.03) + solver -> LBFGS, stepSize -> 0.03) } /** Label to vector converter. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala index 013817a41baf..23e0d45d943a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -80,8 +80,7 @@ private[shared] object SharedParamsCodeGen { " 0)", isValid = "ParamValidators.gt(0)"), ParamDesc[String]("weightCol", "weight column name. If this is not set or empty, we treat " + "all instance weights as 1.0"), - ParamDesc[String]("solver", "the solver algorithm for optimization. If this is not set or " + - "empty, default value is 'auto'", Some("\"auto\"")), + ParamDesc[String]("solver", "the solver algorithm for optimization", finalFields = false), ParamDesc[Int]("aggregationDepth", "suggested depth for treeAggregate (>= 2)", Some("2"), isValid = "ParamValidators.gtEq(2)", isExpertParam = true)) @@ -99,6 +98,7 @@ private[shared] object SharedParamsCodeGen { defaultValueStr: Option[String] = None, isValid: String = "", finalMethods: Boolean = true, + finalFields: Boolean = true, isExpertParam: Boolean = false) { require(name.matches("[a-z][a-zA-Z0-9]*"), s"Param name $name is invalid.") @@ -167,6 +167,11 @@ private[shared] object SharedParamsCodeGen { } else { "def" } + val fieldStr = if (param.finalFields) { + "final val" + } else { + "val" + } val htmlCompliantDoc = Utility.escape(doc) @@ -180,7 +185,7 @@ private[shared] object SharedParamsCodeGen { | * Param for $htmlCompliantDoc. | * @group ${groupStr(0)} | */ - | final val $name: $Param = new $Param(this, "$name", "$doc"$isValid) + | $fieldStr $name: $Param = new $Param(this, "$name", "$doc"$isValid) |$setDefault | /** @group ${groupStr(1)} */ | $methodStr get$Name: $T = $$($name) diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala index 50619607a505..1a8f499798b8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -374,17 +374,15 @@ private[ml] trait HasWeightCol extends Params { } /** - * Trait for shared param solver (default: "auto"). + * Trait for shared param solver. */ private[ml] trait HasSolver extends Params { /** - * Param for the solver algorithm for optimization. If this is not set or empty, default value is 'auto'. + * Param for the solver algorithm for optimization. * @group param */ - final val solver: Param[String] = new Param[String](this, "solver", "the solver algorithm for optimization. If this is not set or empty, default value is 'auto'") - - setDefault(solver, "auto") + val solver: Param[String] = new Param[String](this, "solver", "the solver algorithm for optimization") /** @group getParam */ final def getSolver: String = $(solver) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index ce3460ae4356..c600b87bdc64 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -164,7 +164,18 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam isDefined(linkPredictionCol) && $(linkPredictionCol).nonEmpty } - import GeneralizedLinearRegression._ + /** + * The solver algorithm for optimization. + * Supported options: "irls" (iteratively reweighted least squares). + * Default: "irls" + * + * @group param + */ + @Since("2.3.0") + final override val solver: Param[String] = new Param[String](this, "solver", + "The solver algorithm for optimization. Supported options: " + + s"${supportedSolvers.mkString(", ")}. (Default irls)", + ParamValidators.inArray[String](supportedSolvers)) @Since("2.0.0") override def validateAndTransformSchema( @@ -350,7 +361,7 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val */ @Since("2.0.0") def setSolver(value: String): this.type = set(solver, value) - setDefault(solver -> "irls") + setDefault(solver -> IRLS) /** * Sets the link prediction (linear predictor) column name. @@ -442,6 +453,12 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine Gamma -> Inverse, Gamma -> Identity, Gamma -> Log ) + /** String name for "irls" (iteratively reweighted least squares) solver. */ + private[regression] val IRLS = "irls" + + /** Set of solvers that GeneralizedLinearRegression supports. */ + private[regression] val supportedSolvers = Array(IRLS) + /** Set of family names that GeneralizedLinearRegression supports. */ private[regression] lazy val supportedFamilyNames = supportedFamilyAndLinkPairs.map(_._1.name).toArray :+ "tweedie" diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index db5ac4f14bd3..ce5e0797915d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -34,7 +34,7 @@ import org.apache.spark.ml.optim.WeightedLeastSquares import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.optim.aggregator.LeastSquaresAggregator import org.apache.spark.ml.optim.loss.{L2Regularization, RDDLossFunction} -import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators} import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.evaluation.RegressionMetrics @@ -53,7 +53,23 @@ import org.apache.spark.storage.StorageLevel private[regression] trait LinearRegressionParams extends PredictorParams with HasRegParam with HasElasticNetParam with HasMaxIter with HasTol with HasFitIntercept with HasStandardization with HasWeightCol with HasSolver - with HasAggregationDepth + with HasAggregationDepth { + + import LinearRegression._ + + /** + * The solver algorithm for optimization. + * Supported options: "l-bfgs", "normal" and "auto". + * Default: "auto" + * + * @group param + */ + @Since("2.3.0") + final override val solver: Param[String] = new Param[String](this, "solver", + "The solver algorithm for optimization. Supported options: " + + s"${supportedSolvers.mkString(", ")}. (Default auto)", + ParamValidators.inArray[String](supportedSolvers)) +} /** * Linear regression. @@ -78,6 +94,8 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String extends Regressor[Vector, LinearRegression, LinearRegressionModel] with LinearRegressionParams with DefaultParamsWritable with Logging { + import LinearRegression._ + @Since("1.4.0") def this() = this(Identifiable.randomUID("linReg")) @@ -175,12 +193,8 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String * @group setParam */ @Since("1.6.0") - def setSolver(value: String): this.type = { - require(Set("auto", "l-bfgs", "normal").contains(value), - s"Solver $value was not supported. Supported options: auto, l-bfgs, normal") - set(solver, value) - } - setDefault(solver -> "auto") + def setSolver(value: String): this.type = set(solver, value) + setDefault(solver -> AUTO) /** * Suggested depth for treeAggregate (greater than or equal to 2). @@ -210,8 +224,8 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String elasticNetParam, fitIntercept, maxIter, regParam, standardization, aggregationDepth) instr.logNumFeatures(numFeatures) - if (($(solver) == "auto" && - numFeatures <= WeightedLeastSquares.MAX_NUM_FEATURES) || $(solver) == "normal") { + if (($(solver) == AUTO && + numFeatures <= WeightedLeastSquares.MAX_NUM_FEATURES) || $(solver) == NORMAL) { // For low dimensional data, WeightedLeastSquares is more efficient since the // training algorithm only requires one pass through the data. (SPARK-10668) @@ -444,6 +458,18 @@ object LinearRegression extends DefaultParamsReadable[LinearRegression] { */ @Since("2.1.0") val MAX_FEATURES_FOR_NORMAL_SOLVER: Int = WeightedLeastSquares.MAX_NUM_FEATURES + + /** String name for "auto". */ + private[regression] val AUTO = "auto" + + /** String name for "normal". */ + private[regression] val NORMAL = "normal" + + /** String name for "l-bfgs". */ + private[regression] val LBFGS = "l-bfgs" + + /** Set of solvers that LinearRegression supports. */ + private[regression] val supportedSolvers = Array(AUTO, NORMAL, LBFGS) } /** diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 9b345ac73f3d..948806a5c936 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1265,8 +1265,8 @@ def theta(self): @inherit_doc class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, - HasMaxIter, HasTol, HasSeed, HasStepSize, JavaMLWritable, - JavaMLReadable): + HasMaxIter, HasTol, HasSeed, HasStepSize, HasSolver, + JavaMLWritable, JavaMLReadable): """ Classifier trainer based on the Multilayer Perceptron. Each layer has sigmoid activation function, output layer has softmax. @@ -1407,20 +1407,6 @@ def getStepSize(self): """ return self.getOrDefault(self.stepSize) - @since("2.0.0") - def setSolver(self, value): - """ - Sets the value of :py:attr:`solver`. - """ - return self._set(solver=value) - - @since("2.0.0") - def getSolver(self): - """ - Gets the value of solver or its default value. - """ - return self.getOrDefault(self.solver) - @since("2.0.0") def setInitialWeights(self, value): """ diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 2d17f95b0c44..84d843369e10 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -95,6 +95,9 @@ class LinearRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPrediction .. versionadded:: 1.4.0 """ + solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + + "options: auto, normal, l-bfgs.", typeConverter=TypeConverters.toString) + @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, @@ -1371,6 +1374,8 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha linkPower = Param(Params._dummy(), "linkPower", "The index in the power link function. " + "Only applicable to the Tweedie family.", typeConverter=TypeConverters.toFloat) + solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + + "options: irls.", typeConverter=TypeConverters.toString) @keyword_only def __init__(self, labelCol="label", featuresCol="features", predictionCol="prediction", From 6beca9ce94f484de2f9ffb946bef8334781b3122 Mon Sep 17 00:00:00 2001 From: Devaraj K Date: Sat, 1 Jul 2017 15:53:49 +0100 Subject: [PATCH 012/125] [SPARK-21170][CORE] Utils.tryWithSafeFinallyAndFailureCallbacks throws IllegalArgumentException: Self-suppression not permitted ## What changes were proposed in this pull request? Not adding the exception to the suppressed if it is the same instance as originalThrowable. ## How was this patch tested? Added new tests to verify this, these tests fail without source code changes and passes with the change. Author: Devaraj K Closes #18384 from devaraj-kavali/SPARK-21170. --- .../scala/org/apache/spark/util/Utils.scala | 30 +++---- .../org/apache/spark/util/UtilsSuite.scala | 88 ++++++++++++++++++- 2 files changed, 99 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index bbb7999e2a14..26f61e25da4d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1348,14 +1348,10 @@ private[spark] object Utils extends Logging { try { finallyBlock } catch { - case t: Throwable => - if (originalThrowable != null) { - originalThrowable.addSuppressed(t) - logWarning(s"Suppressing exception in finally: " + t.getMessage, t) - throw originalThrowable - } else { - throw t - } + case t: Throwable if (originalThrowable != null && originalThrowable != t) => + originalThrowable.addSuppressed(t) + logWarning(s"Suppressing exception in finally: ${t.getMessage}", t) + throw originalThrowable } } } @@ -1387,22 +1383,20 @@ private[spark] object Utils extends Logging { catchBlock } catch { case t: Throwable => - originalThrowable.addSuppressed(t) - logWarning(s"Suppressing exception in catch: " + t.getMessage, t) + if (originalThrowable != t) { + originalThrowable.addSuppressed(t) + logWarning(s"Suppressing exception in catch: ${t.getMessage}", t) + } } throw originalThrowable } finally { try { finallyBlock } catch { - case t: Throwable => - if (originalThrowable != null) { - originalThrowable.addSuppressed(t) - logWarning(s"Suppressing exception in finally: " + t.getMessage, t) - throw originalThrowable - } else { - throw t - } + case t: Throwable if (originalThrowable != null && originalThrowable != t) => + originalThrowable.addSuppressed(t) + logWarning(s"Suppressing exception in finally: ${t.getMessage}", t) + throw originalThrowable } } } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index f7bc8f888b0d..4ce143f18bbf 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -38,7 +38,7 @@ import org.apache.commons.math3.stat.inference.ChiSquareTest import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkFunSuite, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.network.util.ByteUnit @@ -1024,4 +1024,90 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { assert(redactedConf("spark.sensitive.property") === Utils.REDACTION_REPLACEMENT_TEXT) } + + test("tryWithSafeFinally") { + var e = new Error("Block0") + val finallyBlockError = new Error("Finally Block") + var isErrorOccurred = false + // if the try and finally blocks throw different exception instances + try { + Utils.tryWithSafeFinally { throw e }(finallyBlock = { throw finallyBlockError }) + } catch { + case t: Error => + assert(t.getSuppressed.head == finallyBlockError) + isErrorOccurred = true + } + assert(isErrorOccurred) + // if the try and finally blocks throw the same exception instance then it should not + // try to add to suppressed and get IllegalArgumentException + e = new Error("Block1") + isErrorOccurred = false + try { + Utils.tryWithSafeFinally { throw e }(finallyBlock = { throw e }) + } catch { + case t: Error => + assert(t.getSuppressed.length == 0) + isErrorOccurred = true + } + assert(isErrorOccurred) + // if the try throws the exception and finally doesn't throw exception + e = new Error("Block2") + isErrorOccurred = false + try { + Utils.tryWithSafeFinally { throw e }(finallyBlock = {}) + } catch { + case t: Error => + assert(t.getSuppressed.length == 0) + isErrorOccurred = true + } + assert(isErrorOccurred) + // if the try and finally block don't throw exception + Utils.tryWithSafeFinally {}(finallyBlock = {}) + } + + test("tryWithSafeFinallyAndFailureCallbacks") { + var e = new Error("Block0") + val catchBlockError = new Error("Catch Block") + val finallyBlockError = new Error("Finally Block") + var isErrorOccurred = false + TaskContext.setTaskContext(TaskContext.empty()) + // if the try, catch and finally blocks throw different exception instances + try { + Utils.tryWithSafeFinallyAndFailureCallbacks { throw e }( + catchBlock = { throw catchBlockError }, finallyBlock = { throw finallyBlockError }) + } catch { + case t: Error => + assert(t.getSuppressed.head == catchBlockError) + assert(t.getSuppressed.last == finallyBlockError) + isErrorOccurred = true + } + assert(isErrorOccurred) + // if the try, catch and finally blocks throw the same exception instance then it should not + // try to add to suppressed and get IllegalArgumentException + e = new Error("Block1") + isErrorOccurred = false + try { + Utils.tryWithSafeFinallyAndFailureCallbacks { throw e }(catchBlock = { throw e }, + finallyBlock = { throw e }) + } catch { + case t: Error => + assert(t.getSuppressed.length == 0) + isErrorOccurred = true + } + assert(isErrorOccurred) + // if the try throws the exception, catch and finally don't throw exceptions + e = new Error("Block2") + isErrorOccurred = false + try { + Utils.tryWithSafeFinallyAndFailureCallbacks { throw e }(catchBlock = {}, finallyBlock = {}) + } catch { + case t: Error => + assert(t.getSuppressed.length == 0) + isErrorOccurred = true + } + assert(isErrorOccurred) + // if the try, catch and finally blocks don't throw exceptions + Utils.tryWithSafeFinallyAndFailureCallbacks {}(catchBlock = {}, finallyBlock = {}) + TaskContext.unset + } } From c605fee01f180588ecb2f48710a7b84073bd3b9a Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Sun, 2 Jul 2017 08:50:48 +0100 Subject: [PATCH 013/125] [SPARK-21260][SQL][MINOR] Remove the unused OutputFakerExec ## What changes were proposed in this pull request? OutputFakerExec was added long ago and is not used anywhere now so we should remove it. ## How was this patch tested? N/A Author: Xingbo Jiang Closes #18473 from jiangxb1987/OutputFakerExec. --- .../spark/sql/execution/basicPhysicalOperators.scala | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index f3ca8397047f..2151c339b9b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -584,17 +584,6 @@ case class CoalesceExec(numPartitions: Int, child: SparkPlan) extends UnaryExecN } } -/** - * A plan node that does nothing but lie about the output of its child. Used to spice a - * (hopefully structurally equivalent) tree from a different optimization sequence into an already - * resolved tree. - */ -case class OutputFakerExec(output: Seq[Attribute], child: SparkPlan) extends SparkPlan { - def children: Seq[SparkPlan] = child :: Nil - - protected override def doExecute(): RDD[InternalRow] = child.execute() -} - /** * Physical plan for a subquery. */ From c19680be1c532dded1e70edce7a981ba28af09ad Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Sun, 2 Jul 2017 16:17:03 +0800 Subject: [PATCH 014/125] [SPARK-19852][PYSPARK][ML] Python StringIndexer supports 'keep' to handle invalid data ## What changes were proposed in this pull request? This PR is to maintain API parity with changes made in SPARK-17498 to support a new option 'keep' in StringIndexer to handle unseen labels or NULL values with PySpark. Note: This is updated version of #17237 , the primary author of this PR is VinceShieh . ## How was this patch tested? Unit tests. Author: VinceShieh Author: Yanbo Liang Closes #18453 from yanboliang/spark-19852. --- python/pyspark/ml/feature.py | 6 ++++++ python/pyspark/ml/tests.py | 21 +++++++++++++++++++++ 2 files changed, 27 insertions(+) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 77de1cc18246..25ad06f682ed 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -2132,6 +2132,12 @@ class StringIndexer(JavaEstimator, HasInputCol, HasOutputCol, HasHandleInvalid, "frequencyDesc, frequencyAsc, alphabetDesc, alphabetAsc.", typeConverter=TypeConverters.toString) + handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid data (unseen " + + "labels or NULL values). Options are 'skip' (filter out rows with " + + "invalid data), error (throw an error), or 'keep' (put invalid data " + + "in a special additional bucket, at index numLabels).", + typeConverter=TypeConverters.toString) + @keyword_only def __init__(self, inputCol=None, outputCol=None, handleInvalid="error", stringOrderType="frequencyDesc"): diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 17a39472e1fe..ffb8b0a890ff 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -551,6 +551,27 @@ def test_rformula_string_indexer_order_type(self): for i in range(0, len(expected)): self.assertTrue(all(observed[i]["features"].toArray() == expected[i])) + def test_string_indexer_handle_invalid(self): + df = self.spark.createDataFrame([ + (0, "a"), + (1, "d"), + (2, None)], ["id", "label"]) + + si1 = StringIndexer(inputCol="label", outputCol="indexed", handleInvalid="keep", + stringOrderType="alphabetAsc") + model1 = si1.fit(df) + td1 = model1.transform(df) + actual1 = td1.select("id", "indexed").collect() + expected1 = [Row(id=0, indexed=0.0), Row(id=1, indexed=1.0), Row(id=2, indexed=2.0)] + self.assertEqual(actual1, expected1) + + si2 = si1.setHandleInvalid("skip") + model2 = si2.fit(df) + td2 = model2.transform(df) + actual2 = td2.select("id", "indexed").collect() + expected2 = [Row(id=0, indexed=0.0), Row(id=1, indexed=1.0)] + self.assertEqual(actual2, expected2) + class HasInducedError(Params): From d4107196d59638845bd19da6aab074424d90ddaf Mon Sep 17 00:00:00 2001 From: Rui Zha Date: Sun, 2 Jul 2017 17:37:47 -0700 Subject: [PATCH 015/125] [SPARK-18004][SQL] Make sure the date or timestamp related predicate can be pushed down to Oracle correctly ## What changes were proposed in this pull request? Move `compileValue` method in JDBCRDD to JdbcDialect, and override the `compileValue` method in OracleDialect to rewrite the Oracle-specific timestamp and date literals in where clause. ## How was this patch tested? An integration test has been added. Author: Rui Zha Author: Zharui Closes #18451 from SharpRay/extend-compileValue-to-dialects. --- .../sql/jdbc/OracleIntegrationSuite.scala | 45 +++++++++++++++++++ .../execution/datasources/jdbc/JDBCRDD.scala | 35 +++++---------- .../apache/spark/sql/jdbc/JdbcDialects.scala | 27 ++++++++++- .../apache/spark/sql/jdbc/OracleDialect.scala | 15 ++++++- 4 files changed, 95 insertions(+), 27 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index b2f096964427..e14810a32edc 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -223,4 +223,49 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo val types = rows(0).toSeq.map(x => x.getClass.toString) assert(types(1).equals("class java.sql.Timestamp")) } + + test("SPARK-18004: Make sure date or timestamp related predicate is pushed down correctly") { + val props = new Properties() + props.put("oracle.jdbc.mapDateToTimestamp", "false") + + val schema = StructType(Seq( + StructField("date_type", DateType, true), + StructField("timestamp_type", TimestampType, true) + )) + + val tableName = "test_date_timestamp_pushdown" + val dateVal = Date.valueOf("2017-06-22") + val timestampVal = Timestamp.valueOf("2017-06-22 21:30:07") + + val data = spark.sparkContext.parallelize(Seq( + Row(dateVal, timestampVal) + )) + + val dfWrite = spark.createDataFrame(data, schema) + dfWrite.write.jdbc(jdbcUrl, tableName, props) + + val dfRead = spark.read.jdbc(jdbcUrl, tableName, props) + + val millis = System.currentTimeMillis() + val dt = new java.sql.Date(millis) + val ts = new java.sql.Timestamp(millis) + + // Query Oracle table with date and timestamp predicates + // which should be pushed down to Oracle. + val df = dfRead.filter(dfRead.col("date_type").lt(dt)) + .filter(dfRead.col("timestamp_type").lt(ts)) + + val metadata = df.queryExecution.sparkPlan.metadata + // The "PushedFilters" part should be exist in Datafrome's + // physical plan and the existence of right literals in + // "PushedFilters" is used to prove that the predicates + // pushing down have been effective. + assert(metadata.get("PushedFilters").ne(None)) + assert(metadata("PushedFilters").contains(dt.toString)) + assert(metadata("PushedFilters").contains(ts.toString)) + + val row = df.collect()(0) + assert(row.getDate(0).equals(dateVal)) + assert(row.getTimestamp(1).equals(timestampVal)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index 2bdc43254133..0f53b5c7c6f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -17,12 +17,10 @@ package org.apache.spark.sql.execution.datasources.jdbc -import java.sql.{Connection, Date, PreparedStatement, ResultSet, SQLException, Timestamp} +import java.sql.{Connection, PreparedStatement, ResultSet, SQLException} import scala.util.control.NonFatal -import org.apache.commons.lang3.StringUtils - import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -86,20 +84,6 @@ object JDBCRDD extends Logging { new StructType(columns.map(name => fieldMap(name))) } - /** - * Converts value to SQL expression. - */ - private def compileValue(value: Any): Any = value match { - case stringValue: String => s"'${escapeSql(stringValue)}'" - case timestampValue: Timestamp => "'" + timestampValue + "'" - case dateValue: Date => "'" + dateValue + "'" - case arrayValue: Array[Any] => arrayValue.map(compileValue).mkString(", ") - case _ => value - } - - private def escapeSql(value: String): String = - if (value == null) null else StringUtils.replace(value, "'", "''") - /** * Turns a single Filter into a String representing a SQL expression. * Returns None for an unhandled filter. @@ -108,15 +92,16 @@ object JDBCRDD extends Logging { def quote(colName: String): String = dialect.quoteIdentifier(colName) Option(f match { - case EqualTo(attr, value) => s"${quote(attr)} = ${compileValue(value)}" + case EqualTo(attr, value) => s"${quote(attr)} = ${dialect.compileValue(value)}" case EqualNullSafe(attr, value) => val col = quote(attr) - s"(NOT ($col != ${compileValue(value)} OR $col IS NULL OR " + - s"${compileValue(value)} IS NULL) OR ($col IS NULL AND ${compileValue(value)} IS NULL))" - case LessThan(attr, value) => s"${quote(attr)} < ${compileValue(value)}" - case GreaterThan(attr, value) => s"${quote(attr)} > ${compileValue(value)}" - case LessThanOrEqual(attr, value) => s"${quote(attr)} <= ${compileValue(value)}" - case GreaterThanOrEqual(attr, value) => s"${quote(attr)} >= ${compileValue(value)}" + s"(NOT ($col != ${dialect.compileValue(value)} OR $col IS NULL OR " + + s"${dialect.compileValue(value)} IS NULL) OR " + + s"($col IS NULL AND ${dialect.compileValue(value)} IS NULL))" + case LessThan(attr, value) => s"${quote(attr)} < ${dialect.compileValue(value)}" + case GreaterThan(attr, value) => s"${quote(attr)} > ${dialect.compileValue(value)}" + case LessThanOrEqual(attr, value) => s"${quote(attr)} <= ${dialect.compileValue(value)}" + case GreaterThanOrEqual(attr, value) => s"${quote(attr)} >= ${dialect.compileValue(value)}" case IsNull(attr) => s"${quote(attr)} IS NULL" case IsNotNull(attr) => s"${quote(attr)} IS NOT NULL" case StringStartsWith(attr, value) => s"${quote(attr)} LIKE '${value}%'" @@ -124,7 +109,7 @@ object JDBCRDD extends Logging { case StringContains(attr, value) => s"${quote(attr)} LIKE '%${value}%'" case In(attr, value) if value.isEmpty => s"CASE WHEN ${quote(attr)} IS NULL THEN NULL ELSE FALSE END" - case In(attr, value) => s"${quote(attr)} IN (${compileValue(value)})" + case In(attr, value) => s"${quote(attr)} IN (${dialect.compileValue(value)})" case Not(f) => compileFilter(f, dialect).map(p => s"(NOT ($p))").getOrElse(null) case Or(f1, f2) => // We can't compile Or filter unless both sub-filters are compiled successfully. 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 a86a86d40890..7c38ed68c041 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 @@ -17,7 +17,9 @@ package org.apache.spark.sql.jdbc -import java.sql.Connection +import java.sql.{Connection, Date, Timestamp} + +import org.apache.commons.lang3.StringUtils import org.apache.spark.annotation.{DeveloperApi, InterfaceStability, Since} import org.apache.spark.sql.types._ @@ -123,6 +125,29 @@ abstract class JdbcDialect extends Serializable { def beforeFetch(connection: Connection, properties: Map[String, String]): Unit = { } + /** + * Escape special characters in SQL string literals. + * @param value The string to be escaped. + * @return Escaped string. + */ + @Since("2.3.0") + protected[jdbc] def escapeSql(value: String): String = + if (value == null) null else StringUtils.replace(value, "'", "''") + + /** + * Converts value to SQL expression. + * @param value The value to be converted. + * @return Converted value. + */ + @Since("2.3.0") + def compileValue(value: Any): Any = value match { + case stringValue: String => s"'${escapeSql(stringValue)}'" + case timestampValue: Timestamp => "'" + timestampValue + "'" + case dateValue: Date => "'" + dateValue + "'" + case arrayValue: Array[Any] => arrayValue.map(compileValue).mkString(", ") + case _ => value + } + /** * Return Some[true] iff `TRUNCATE TABLE` causes cascading default. * Some[true] : TRUNCATE TABLE causes cascading. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala index 20e634c06b61..3b44c1de93a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.jdbc -import java.sql.Types +import java.sql.{Date, Timestamp, Types} import org.apache.spark.sql.types._ @@ -64,5 +64,18 @@ private case object OracleDialect extends JdbcDialect { case _ => None } + override def compileValue(value: Any): Any = value match { + // The JDBC drivers support date literals in SQL statements written in the + // format: {d 'yyyy-mm-dd'} and timestamp literals in SQL statements written + // in the format: {ts 'yyyy-mm-dd hh:mm:ss.f...'}. For details, see + // 'Oracle Database JDBC Developer’s Guide and Reference, 11g Release 1 (11.1)' + // Appendix A Reference Information. + case stringValue: String => s"'${escapeSql(stringValue)}'" + case timestampValue: Timestamp => "{ts '" + timestampValue + "'}" + case dateValue: Date => "{d '" + dateValue + "'}" + case arrayValue: Array[Any] => arrayValue.map(compileValue).mkString(", ") + case _ => value + } + override def isCascadingTruncateTable(): Option[Boolean] = Some(false) } From d913db16a0de0983961f9d0c5f9b146be7226ac1 Mon Sep 17 00:00:00 2001 From: guoxiaolong Date: Mon, 3 Jul 2017 13:31:01 +0800 Subject: [PATCH 016/125] [SPARK-21250][WEB-UI] Add a url in the table of 'Running Executors' in worker page to visit job page. ## What changes were proposed in this pull request? Add a url in the table of 'Running Executors' in worker page to visit job page. When I click URL of 'Name', the current page jumps to the job page. Of course this is only in the table of 'Running Executors'. This URL of 'Name' is in the table of 'Finished Executors' does not exist, the click will not jump to any page. fix before: ![1](https://user-images.githubusercontent.com/26266482/27679397-30ddc262-5ceb-11e7-839b-0889d1f42480.png) fix after: ![2](https://user-images.githubusercontent.com/26266482/27679405-3588ef12-5ceb-11e7-9756-0a93815cd698.png) ## How was this patch tested? manual tests Please review http://spark.apache.org/contributing.html before opening a pull request. Author: guoxiaolong Closes #18464 from guoxiaolongzte/SPARK-21250. --- .../apache/spark/deploy/worker/ui/WorkerPage.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala index 1ad973122b60..ea39b0dce0a4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala @@ -23,8 +23,8 @@ import scala.xml.Node import org.json4s.JValue +import org.apache.spark.deploy.{ExecutorState, JsonProtocol} import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} -import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} import org.apache.spark.ui.{UIUtils, WebUIPage} @@ -112,7 +112,15 @@ private[ui] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") {
  • ID: {executor.appId}
  • -
  • Name: {executor.appDesc.name}
  • +
  • Name: + { + if ({executor.state == ExecutorState.RUNNING} && executor.appDesc.appUiUrl.nonEmpty) { + {executor.appDesc.name} + } else { + {executor.appDesc.name} + } + } +
  • User: {executor.appDesc.user}
From a9339db99f0620d4828eb903523be55dfbf2fb64 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 3 Jul 2017 19:52:39 +0800 Subject: [PATCH 017/125] [SPARK-21137][CORE] Spark reads many small files slowly ## What changes were proposed in this pull request? Parallelize FileInputFormat.listStatus in Hadoop API via LIST_STATUS_NUM_THREADS to speed up examination of file sizes for wholeTextFiles et al ## How was this patch tested? Existing tests, which will exercise the key path here: using a local file system. Author: Sean Owen Closes #18441 from srowen/SPARK-21137. --- .../main/scala/org/apache/spark/rdd/BinaryFileRDD.scala | 7 ++++++- .../main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala | 7 ++++++- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index 50d977a92da5..a14bad47dfe1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.task.JobContextImpl import org.apache.spark.{Partition, SparkContext} @@ -35,8 +36,12 @@ private[spark] class BinaryFileRDD[T]( extends NewHadoopRDD[String, T](sc, inputFormatClass, keyClass, valueClass, conf) { override def getPartitions: Array[Partition] = { - val inputFormat = inputFormatClass.newInstance val conf = getConf + // setMinPartitions below will call FileInputFormat.listStatus(), which can be quite slow when + // traversing a large number of directories and files. Parallelize it. + conf.setIfUnset(FileInputFormat.LIST_STATUS_NUM_THREADS, + Runtime.getRuntime.availableProcessors().toString) + val inputFormat = inputFormatClass.newInstance inputFormat match { case configurable: Configurable => configurable.setConf(conf) diff --git a/core/src/main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala index 8e1baae796fc..9f3d0745c33c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.{Text, Writable} import org.apache.hadoop.mapreduce.InputSplit +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.task.JobContextImpl import org.apache.spark.{Partition, SparkContext} @@ -38,8 +39,12 @@ private[spark] class WholeTextFileRDD( extends NewHadoopRDD[Text, Text](sc, inputFormatClass, keyClass, valueClass, conf) { override def getPartitions: Array[Partition] = { - val inputFormat = inputFormatClass.newInstance val conf = getConf + // setMinPartitions below will call FileInputFormat.listStatus(), which can be quite slow when + // traversing a large number of directories and files. Parallelize it. + conf.setIfUnset(FileInputFormat.LIST_STATUS_NUM_THREADS, + Runtime.getRuntime.availableProcessors().toString) + val inputFormat = inputFormatClass.newInstance inputFormat match { case configurable: Configurable => configurable.setConf(conf) From eb7a5a66bbd5837c01f13c76b68de2a6034976f3 Mon Sep 17 00:00:00 2001 From: Zhenhua Wang Date: Mon, 3 Jul 2017 09:01:42 -0700 Subject: [PATCH 018/125] [TEST] Load test table based on case sensitivity ## What changes were proposed in this pull request? It is strange that we will get "table not found" error if **the first sql** uses upper case table names, when developers write tests with `TestHiveSingleton`, **although case insensitivity**. This is because in `TestHiveQueryExecution`, test tables are loaded based on exact matching instead of case sensitivity. ## How was this patch tested? Added a new test case. Author: Zhenhua Wang Closes #18504 from wzhfy/testHive. --- .../apache/spark/sql/hive/test/TestHive.scala | 7 ++- .../apache/spark/sql/hive/TestHiveSuite.scala | 45 +++++++++++++++++++ 2 files changed, 51 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/TestHiveSuite.scala 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 4e1792321c89..801f9b992364 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 @@ -449,6 +449,8 @@ private[hive] class TestHiveSparkSession( private val loadedTables = new collection.mutable.HashSet[String] + def getLoadedTables: collection.mutable.HashSet[String] = loadedTables + def loadTestTable(name: String) { if (!(loadedTables contains name)) { // Marks the table as loaded first to prevent infinite mutually recursive table loading. @@ -553,7 +555,10 @@ private[hive] class TestHiveQueryExecution( val referencedTables = describedTables ++ logical.collect { case UnresolvedRelation(tableIdent, _) => tableIdent.table } - val referencedTestTables = referencedTables.filter(sparkSession.testTables.contains) + val resolver = sparkSession.sessionState.conf.resolver + val referencedTestTables = sparkSession.testTables.keys.filter { testTable => + referencedTables.exists(resolver(_, testTable)) + } logDebug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") referencedTestTables.foreach(sparkSession.loadTestTable) // Proceed with analysis. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/TestHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/TestHiveSuite.scala new file mode 100644 index 000000000000..193fa83dbad9 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/TestHiveSuite.scala @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.AnalysisException +import org.apache.spark.sql.hive.test.{TestHiveSingleton, TestHiveSparkSession} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils + + +class TestHiveSuite extends TestHiveSingleton with SQLTestUtils { + test("load test table based on case sensitivity") { + val testHiveSparkSession = spark.asInstanceOf[TestHiveSparkSession] + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + sql("SELECT * FROM SRC").queryExecution.analyzed + assert(testHiveSparkSession.getLoadedTables.contains("src")) + assert(testHiveSparkSession.getLoadedTables.size == 1) + } + testHiveSparkSession.reset() + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val err = intercept[AnalysisException] { + sql("SELECT * FROM SRC").queryExecution.analyzed + } + assert(err.message.contains("Table or view not found")) + } + testHiveSparkSession.reset() + } +} From 17bdc36ef16a544b693c628db276fe32db87fe7a Mon Sep 17 00:00:00 2001 From: aokolnychyi Date: Mon, 3 Jul 2017 09:35:49 -0700 Subject: [PATCH 019/125] [SPARK-21102][SQL] Refresh command is too aggressive in parsing ### Idea This PR adds validation to REFRESH sql statements. Currently, users can specify whatever they want as resource path. For example, spark.sql("REFRESH ! $ !") will be executed without any exceptions. ### Implementation I am not sure that my current implementation is the most optimal, so any feedback is appreciated. My first idea was to make the grammar as strict as possible. Unfortunately, there were some problems. I tried the approach below: SqlBase.g4 ``` ... | REFRESH TABLE tableIdentifier #refreshTable | REFRESH resourcePath #refreshResource ... resourcePath : STRING | (IDENTIFIER | number | nonReserved | '/' | '-')+ // other symbols can be added if needed ; ``` It is not flexible enough and requires to explicitly mention all possible symbols. Therefore, I came up with the current approach that is implemented in the code. Let me know your opinion on which one is better. Author: aokolnychyi Closes #18368 from aokolnychyi/spark-21102. --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../spark/sql/execution/SparkSqlParser.scala | 20 +++++++++++++++--- .../sql/execution/SparkSqlParserSuite.scala | 21 ++++++++++++++++++- 3 files changed, 38 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 7ffa15009633..29f554451ed4 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -149,7 +149,7 @@ statement | (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)? tableIdentifier partitionSpec? describeColName? #describeTable | REFRESH TABLE tableIdentifier #refreshTable - | REFRESH .*? #refreshResource + | REFRESH (STRING | .*?) #refreshResource | CACHE LAZY? TABLE tableIdentifier (AS? query)? #cacheTable | UNCACHE TABLE (IF EXISTS)? tableIdentifier #uncacheTable | CLEAR CACHE #clearCache diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 3c58c6e1b678..2b79eb5eac0f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -230,11 +230,25 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } /** - * Create a [[RefreshTable]] logical plan. + * Create a [[RefreshResource]] logical plan. */ override def visitRefreshResource(ctx: RefreshResourceContext): LogicalPlan = withOrigin(ctx) { - val resourcePath = remainder(ctx.REFRESH.getSymbol).trim - RefreshResource(resourcePath) + val path = if (ctx.STRING != null) string(ctx.STRING) else extractUnquotedResourcePath(ctx) + RefreshResource(path) + } + + private def extractUnquotedResourcePath(ctx: RefreshResourceContext): String = withOrigin(ctx) { + val unquotedPath = remainder(ctx.REFRESH.getSymbol).trim + validate( + unquotedPath != null && !unquotedPath.isEmpty, + "Resource paths cannot be empty in REFRESH statements. Use / to match everything", + ctx) + val forbiddenSymbols = Seq(" ", "\n", "\r", "\t") + validate( + !forbiddenSymbols.exists(unquotedPath.contains(_)), + "REFRESH statements cannot contain ' ', '\\n', '\\r', '\\t' inside unquoted resource paths", + ctx) + unquotedPath } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index bd9c2ebd6fab..d238c76fbeef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, Concat, SortOrder} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RepartitionByExpression, Sort} import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources.CreateTable +import org.apache.spark.sql.execution.datasources.{CreateTable, RefreshResource} import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} @@ -66,6 +66,25 @@ class SparkSqlParserSuite extends AnalysisTest { } } + test("refresh resource") { + assertEqual("REFRESH prefix_path", RefreshResource("prefix_path")) + assertEqual("REFRESH /", RefreshResource("/")) + assertEqual("REFRESH /path///a", RefreshResource("/path///a")) + assertEqual("REFRESH pat1h/112/_1a", RefreshResource("pat1h/112/_1a")) + assertEqual("REFRESH pat1h/112/_1a/a-1", RefreshResource("pat1h/112/_1a/a-1")) + assertEqual("REFRESH path-with-dash", RefreshResource("path-with-dash")) + assertEqual("REFRESH \'path with space\'", RefreshResource("path with space")) + assertEqual("REFRESH \"path with space 2\"", RefreshResource("path with space 2")) + intercept("REFRESH a b", "REFRESH statements cannot contain") + intercept("REFRESH a\tb", "REFRESH statements cannot contain") + intercept("REFRESH a\nb", "REFRESH statements cannot contain") + intercept("REFRESH a\rb", "REFRESH statements cannot contain") + intercept("REFRESH a\r\nb", "REFRESH statements cannot contain") + intercept("REFRESH @ $a$", "REFRESH statements cannot contain") + intercept("REFRESH ", "Resource paths cannot be empty in REFRESH statements") + intercept("REFRESH", "Resource paths cannot be empty in REFRESH statements") + } + test("show functions") { assertEqual("show functions", ShowFunctionsCommand(None, None, true, true)) assertEqual("show all functions", ShowFunctionsCommand(None, None, true, true)) From 363bfe30ba44852a8fac946a37032f76480f6f1b Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Mon, 3 Jul 2017 10:14:03 -0700 Subject: [PATCH 020/125] [SPARK-20073][SQL] Prints an explicit warning message in case of NULL-safe equals ## What changes were proposed in this pull request? This pr added code to print the same warning messages with `===` cases when using NULL-safe equals (`<=>`). ## How was this patch tested? Existing tests. Author: Takeshi Yamamuro Closes #18436 from maropu/SPARK-20073. --- .../src/main/scala/org/apache/spark/sql/Column.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) 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 7e1f1d83cb3d..bd1669b6dba6 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 @@ -464,7 +464,15 @@ class Column(val expr: Expression) extends Logging { * @group expr_ops * @since 1.3.0 */ - def <=> (other: Any): Column = withExpr { EqualNullSafe(expr, lit(other).expr) } + def <=> (other: Any): Column = withExpr { + val right = lit(other).expr + if (this.expr == right) { + logWarning( + s"Constructing trivially true equals predicate, '${this.expr} <=> $right'. " + + "Perhaps you need to use aliases.") + } + EqualNullSafe(expr, right) + } /** * Equality test that is safe for null values. From f953ca56eccdaef29ac580d44613a028415ba3f5 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 3 Jul 2017 10:51:44 -0700 Subject: [PATCH 021/125] [SPARK-21284][SQL] rename SessionCatalog.registerFunction parameter name ## What changes were proposed in this pull request? Looking at the code in `SessionCatalog.registerFunction`, the parameter `ignoreIfExists` is a wrong name. When `ignoreIfExists` is true, we will override the function if it already exists. So `overrideIfExists` should be the corrected name. ## How was this patch tested? N/A Author: Wenchen Fan Closes #18510 from cloud-fan/minor. --- .../sql/catalyst/catalog/SessionCatalog.scala | 6 +++--- .../catalog/SessionCatalogSuite.scala | 20 ++++++++++--------- .../sql/execution/command/functions.scala | 3 +-- .../spark/sql/internal/CatalogSuite.scala | 2 +- .../spark/sql/hive/HiveSessionCatalog.scala | 2 +- .../ObjectHashAggregateExecBenchmark.scala | 2 +- 6 files changed, 18 insertions(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 7ece77df7fc1..a86604e4353a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -1104,10 +1104,10 @@ class SessionCatalog( */ def registerFunction( funcDefinition: CatalogFunction, - ignoreIfExists: Boolean, + overrideIfExists: Boolean, functionBuilder: Option[FunctionBuilder] = None): Unit = { val func = funcDefinition.identifier - if (functionRegistry.functionExists(func) && !ignoreIfExists) { + if (functionRegistry.functionExists(func) && !overrideIfExists) { throw new AnalysisException(s"Function $func already exists") } val info = new ExpressionInfo(funcDefinition.className, func.database.orNull, func.funcName) @@ -1219,7 +1219,7 @@ class SessionCatalog( // catalog. So, it is possible that qualifiedName is not exactly the same as // catalogFunction.identifier.unquotedString (difference is on case-sensitivity). // At here, we preserve the input from the user. - registerFunction(catalogFunction.copy(identifier = qualifiedName), ignoreIfExists = false) + registerFunction(catalogFunction.copy(identifier = qualifiedName), overrideIfExists = false) // Now, we need to create the Expression. functionRegistry.lookupFunction(qualifiedName, children) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index fc3893e19779..8f856a0daad1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -1175,9 +1175,9 @@ abstract class SessionCatalogSuite extends AnalysisTest { val tempFunc1 = (e: Seq[Expression]) => e.head val tempFunc2 = (e: Seq[Expression]) => e.last catalog.registerFunction( - newFunc("temp1", None), ignoreIfExists = false, functionBuilder = Some(tempFunc1)) + newFunc("temp1", None), overrideIfExists = false, functionBuilder = Some(tempFunc1)) catalog.registerFunction( - newFunc("temp2", None), ignoreIfExists = false, functionBuilder = Some(tempFunc2)) + newFunc("temp2", None), overrideIfExists = false, functionBuilder = Some(tempFunc2)) val arguments = Seq(Literal(1), Literal(2), Literal(3)) assert(catalog.lookupFunction(FunctionIdentifier("temp1"), arguments) === Literal(1)) assert(catalog.lookupFunction(FunctionIdentifier("temp2"), arguments) === Literal(3)) @@ -1189,12 +1189,12 @@ abstract class SessionCatalogSuite extends AnalysisTest { // Temporary function already exists val e = intercept[AnalysisException] { catalog.registerFunction( - newFunc("temp1", None), ignoreIfExists = false, functionBuilder = Some(tempFunc3)) + newFunc("temp1", None), overrideIfExists = false, functionBuilder = Some(tempFunc3)) }.getMessage assert(e.contains("Function temp1 already exists")) // Temporary function is overridden catalog.registerFunction( - newFunc("temp1", None), ignoreIfExists = true, functionBuilder = Some(tempFunc3)) + newFunc("temp1", None), overrideIfExists = true, functionBuilder = Some(tempFunc3)) assert( catalog.lookupFunction( FunctionIdentifier("temp1"), arguments) === Literal(arguments.length)) @@ -1208,7 +1208,7 @@ abstract class SessionCatalogSuite extends AnalysisTest { val tempFunc1 = (e: Seq[Expression]) => e.head catalog.registerFunction( - newFunc("temp1", None), ignoreIfExists = false, functionBuilder = Some(tempFunc1)) + newFunc("temp1", None), overrideIfExists = false, functionBuilder = Some(tempFunc1)) // Returns true when the function is temporary assert(catalog.isTemporaryFunction(FunctionIdentifier("temp1"))) @@ -1259,7 +1259,7 @@ abstract class SessionCatalogSuite extends AnalysisTest { withBasicCatalog { catalog => val tempFunc = (e: Seq[Expression]) => e.head catalog.registerFunction( - newFunc("func1", None), ignoreIfExists = false, functionBuilder = Some(tempFunc)) + newFunc("func1", None), overrideIfExists = false, functionBuilder = Some(tempFunc)) val arguments = Seq(Literal(1), Literal(2), Literal(3)) assert(catalog.lookupFunction(FunctionIdentifier("func1"), arguments) === Literal(1)) catalog.dropTempFunction("func1", ignoreIfNotExists = false) @@ -1300,7 +1300,7 @@ abstract class SessionCatalogSuite extends AnalysisTest { withBasicCatalog { catalog => val tempFunc1 = (e: Seq[Expression]) => e.head catalog.registerFunction( - newFunc("func1", None), ignoreIfExists = false, functionBuilder = Some(tempFunc1)) + newFunc("func1", None), overrideIfExists = false, functionBuilder = Some(tempFunc1)) assert(catalog.lookupFunction( FunctionIdentifier("func1"), Seq(Literal(1), Literal(2), Literal(3))) == Literal(1)) catalog.dropTempFunction("func1", ignoreIfNotExists = false) @@ -1318,8 +1318,10 @@ abstract class SessionCatalogSuite extends AnalysisTest { val tempFunc2 = (e: Seq[Expression]) => e.last catalog.createFunction(newFunc("func2", Some("db2")), ignoreIfExists = false) catalog.createFunction(newFunc("not_me", Some("db2")), ignoreIfExists = false) - catalog.registerFunction(funcMeta1, ignoreIfExists = false, functionBuilder = Some(tempFunc1)) - catalog.registerFunction(funcMeta2, ignoreIfExists = false, functionBuilder = Some(tempFunc2)) + catalog.registerFunction( + funcMeta1, overrideIfExists = false, functionBuilder = Some(tempFunc1)) + catalog.registerFunction( + funcMeta2, overrideIfExists = false, functionBuilder = Some(tempFunc2)) assert(catalog.listFunctions("db1", "*").map(_._1).toSet == Set(FunctionIdentifier("func1"), FunctionIdentifier("yes_me"))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala index f39a3269efaf..a91ad413f4d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala @@ -58,9 +58,8 @@ case class CreateFunctionCommand( s"is not allowed: '${databaseName.get}'") } // We first load resources and then put the builder in the function registry. - // Please note that it is allowed to overwrite an existing temp function. catalog.loadFunctionResources(resources) - catalog.registerFunction(func, ignoreIfExists = false) + catalog.registerFunction(func, overrideIfExists = false) } else { // For a permanent, we will store the metadata into underlying external catalog. // This function will be loaded into the FunctionRegistry when a query uses it. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index b2d568ce320e..6acac1a9aa31 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -79,7 +79,7 @@ class CatalogSuite val tempFunc = (e: Seq[Expression]) => e.head val funcMeta = CatalogFunction(FunctionIdentifier(name, None), "className", Nil) sessionCatalog.registerFunction( - funcMeta, ignoreIfExists = false, functionBuilder = Some(tempFunc)) + funcMeta, overrideIfExists = false, functionBuilder = Some(tempFunc)) } private def dropFunction(name: String, db: Option[String] = None): Unit = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index da87f0218e3a..0d0269f69430 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -161,7 +161,7 @@ private[sql] class HiveSessionCatalog( FunctionIdentifier(functionName.toLowerCase(Locale.ROOT), database) val func = CatalogFunction(functionIdentifier, className, Nil) // Put this Hive built-in function to our function registry. - registerFunction(func, ignoreIfExists = false) + registerFunction(func, overrideIfExists = false) // Now, we need to create the Expression. functionRegistry.lookupFunction(functionIdentifier, children) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala index 73383ae4d411..e599d1ab1d48 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala @@ -221,7 +221,7 @@ class ObjectHashAggregateExecBenchmark extends BenchmarkBase with TestHiveSingle val sessionCatalog = sparkSession.sessionState.catalog.asInstanceOf[HiveSessionCatalog] val functionIdentifier = FunctionIdentifier(functionName, database = None) val func = CatalogFunction(functionIdentifier, clazz.getName, resources = Nil) - sessionCatalog.registerFunction(func, ignoreIfExists = false) + sessionCatalog.registerFunction(func, overrideIfExists = false) } private def percentile_approx( From c79c10ebaf3d63b697b8d6d1a7e55aa2d406af69 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 3 Jul 2017 16:18:54 -0700 Subject: [PATCH 022/125] [TEST] Different behaviors of SparkContext Conf when building SparkSession ## What changes were proposed in this pull request? If the created ACTIVE sparkContext is not EXPLICITLY passed through the Builder's API `sparkContext()`, the conf of this sparkContext will also contain the conf set through the API `config()`; otherwise, the conf of this sparkContext will NOT contain the conf set through the API `config()` ## How was this patch tested? N/A Author: gatorsmile Closes #18517 from gatorsmile/fixTestCase2. --- .../spark/sql/SparkSessionBuilderSuite.scala | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index 386d13d07a95..4f6d5f79d466 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -98,12 +98,31 @@ class SparkSessionBuilderSuite extends SparkFunSuite { val session = SparkSession.builder().config("key2", "value2").getOrCreate() assert(session.conf.get("key1") == "value1") assert(session.conf.get("key2") == "value2") + assert(session.sparkContext == sparkContext2) assert(session.sparkContext.conf.get("key1") == "value1") + // If the created sparkContext is not passed through the Builder's API sparkContext, + // the conf of this sparkContext will also contain the conf set through the API config. assert(session.sparkContext.conf.get("key2") == "value2") assert(session.sparkContext.conf.get("spark.app.name") == "test") session.stop() } + test("create SparkContext first then pass context to SparkSession") { + sparkContext.stop() + val conf = new SparkConf().setAppName("test").setMaster("local").set("key1", "value1") + val newSC = new SparkContext(conf) + val session = SparkSession.builder().sparkContext(newSC).config("key2", "value2").getOrCreate() + assert(session.conf.get("key1") == "value1") + assert(session.conf.get("key2") == "value2") + assert(session.sparkContext == newSC) + assert(session.sparkContext.conf.get("key1") == "value1") + // If the created sparkContext is passed through the Builder's API sparkContext, + // the conf of this sparkContext will not contain the conf set through the API config. + assert(!session.sparkContext.conf.contains("key2")) + assert(session.sparkContext.conf.get("spark.app.name") == "test") + session.stop() + } + test("SPARK-15887: hive-site.xml should be loaded") { val session = SparkSession.builder().master("local").getOrCreate() assert(session.sessionState.newHadoopConf().get("hive.in.test") == "true") From 6657e00de36b59011d3fe78e8613fb64e54c957a Mon Sep 17 00:00:00 2001 From: liuxian Date: Tue, 4 Jul 2017 09:16:40 +0800 Subject: [PATCH 023/125] [SPARK-21283][CORE] FileOutputStream should be created as append mode ## What changes were proposed in this pull request? `FileAppender` is used to write `stderr` and `stdout` files in `ExecutorRunner`, But before writing `ErrorStream` into the the `stderr` file, the header information has been written into ,if FileOutputStream is not created as append mode, the header information will be lost ## How was this patch tested? unit test case Author: liuxian Closes #18507 from 10110346/wip-lx-0703. --- .../scala/org/apache/spark/util/logging/FileAppender.scala | 2 +- .../test/scala/org/apache/spark/util/FileAppenderSuite.scala | 5 ++++- 2 files changed, 5 insertions(+), 2 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 fdb1495899bc..8a0cc709bccc 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 @@ -94,7 +94,7 @@ private[spark] class FileAppender(inputStream: InputStream, file: File, bufferSi /** Open the file output stream */ protected def openFile() { - outputStream = new FileOutputStream(file, false) + outputStream = new FileOutputStream(file, true) logDebug(s"Opened file $file") } 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 7e2da8e14153..cd0ed5b036bf 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -52,10 +52,13 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { test("basic file appender") { val testString = (1 to 1000).mkString(", ") val inputStream = new ByteArrayInputStream(testString.getBytes(StandardCharsets.UTF_8)) + // The `header` should not be covered + val header = "Add header" + Files.write(header, testFile, StandardCharsets.UTF_8) val appender = new FileAppender(inputStream, testFile) inputStream.close() appender.awaitTermination() - assert(Files.toString(testFile, StandardCharsets.UTF_8) === testString) + assert(Files.toString(testFile, StandardCharsets.UTF_8) === header + testString) } test("rolling file appender - time-based rolling") { From a848d552ef6b5d0d3bb3b2da903478437a8b10aa Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Tue, 4 Jul 2017 11:35:08 +0900 Subject: [PATCH 024/125] [SPARK-21264][PYTHON] Call cross join path in join without 'on' and with 'how' ## What changes were proposed in this pull request? Currently, it throws a NPE when missing columns but join type is speicified in join at PySpark as below: ```python spark.conf.set("spark.sql.crossJoin.enabled", "false") spark.range(1).join(spark.range(1), how="inner").show() ``` ``` Traceback (most recent call last): ... py4j.protocol.Py4JJavaError: An error occurred while calling o66.join. : java.lang.NullPointerException at org.apache.spark.sql.Dataset.join(Dataset.scala:931) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) ... ``` ```python spark.conf.set("spark.sql.crossJoin.enabled", "true") spark.range(1).join(spark.range(1), how="inner").show() ``` ``` ... py4j.protocol.Py4JJavaError: An error occurred while calling o84.join. : java.lang.NullPointerException at org.apache.spark.sql.Dataset.join(Dataset.scala:931) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) ... ``` This PR suggests to follow Scala's one as below: ```scala scala> spark.conf.set("spark.sql.crossJoin.enabled", "false") scala> spark.range(1).join(spark.range(1), Seq.empty[String], "inner").show() ``` ``` org.apache.spark.sql.AnalysisException: Detected cartesian product for INNER join between logical plans Range (0, 1, step=1, splits=Some(8)) and Range (0, 1, step=1, splits=Some(8)) Join condition is missing or trivial. Use the CROSS JOIN syntax to allow cartesian products between these relations.; ... ``` ```scala scala> spark.conf.set("spark.sql.crossJoin.enabled", "true") scala> spark.range(1).join(spark.range(1), Seq.empty[String], "inner").show() ``` ``` +---+---+ | id| id| +---+---+ | 0| 0| +---+---+ ``` **After** ```python spark.conf.set("spark.sql.crossJoin.enabled", "false") spark.range(1).join(spark.range(1), how="inner").show() ``` ``` Traceback (most recent call last): ... pyspark.sql.utils.AnalysisException: u'Detected cartesian product for INNER join between logical plans\nRange (0, 1, step=1, splits=Some(8))\nand\nRange (0, 1, step=1, splits=Some(8))\nJoin condition is missing or trivial.\nUse the CROSS JOIN syntax to allow cartesian products between these relations.;' ``` ```python spark.conf.set("spark.sql.crossJoin.enabled", "true") spark.range(1).join(spark.range(1), how="inner").show() ``` ``` +---+---+ | id| id| +---+---+ | 0| 0| +---+---+ ``` ## How was this patch tested? Added tests in `python/pyspark/sql/tests.py`. Author: hyukjinkwon Closes #18484 from HyukjinKwon/SPARK-21264. --- python/pyspark/sql/dataframe.py | 2 ++ python/pyspark/sql/tests.py | 16 ++++++++++++++++ 2 files changed, 18 insertions(+) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 0649271ed224..27a6dad8917d 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -833,6 +833,8 @@ def join(self, other, on=None, how=None): else: if how is None: how = "inner" + if on is None: + on = self._jseq([]) assert isinstance(how, basestring), "how should be basestring" jdf = self._jdf.join(other._jdf, on, how) return DataFrame(jdf, self.sql_ctx) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 0a1cd6856b8e..c105969b26b9 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -2021,6 +2021,22 @@ def test_toDF_with_schema_string(self): self.assertEqual(df.schema.simpleString(), "struct") self.assertEqual(df.collect(), [Row(key=i) for i in range(100)]) + def test_join_without_on(self): + df1 = self.spark.range(1).toDF("a") + df2 = self.spark.range(1).toDF("b") + + try: + self.spark.conf.set("spark.sql.crossJoin.enabled", "false") + self.assertRaises(AnalysisException, lambda: df1.join(df2, how="inner").collect()) + + self.spark.conf.set("spark.sql.crossJoin.enabled", "true") + actual = df1.join(df2, how="inner").collect() + expected = [Row(a=0, b=0)] + self.assertEqual(actual, expected) + finally: + # We should unset this. Otherwise, other tests are affected. + self.spark.conf.unset("spark.sql.crossJoin.enabled") + # Regression test for invalid join methods when on is None, Spark-14761 def test_invalid_join_method(self): df1 = self.spark.createDataFrame([("Alice", 5), ("Bob", 8)], ["name", "age"]) From 8ca4ebefa6301d9cb633ea15cf71f49c2d7f8607 Mon Sep 17 00:00:00 2001 From: Thomas Decaux Date: Tue, 4 Jul 2017 12:17:48 +0100 Subject: [PATCH 025/125] [MINOR] Add french stop word "les" ## What changes were proposed in this pull request? Added "les" as french stop word (plurial of le) Author: Thomas Decaux Closes #18514 from ebuildy/patch-1. --- .../resources/org/apache/spark/ml/feature/stopwords/french.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/french.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/french.txt index 94b8f8f39a3e..a59a0424616c 100644 --- a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/french.txt +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/french.txt @@ -15,6 +15,7 @@ il je la le +les leur lui ma @@ -152,4 +153,4 @@ eusses eût eussions eussiez -eussent \ No newline at end of file +eussent From 2b1e94b9add82b30bc94f639fa97492624bf0dce Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Tue, 4 Jul 2017 12:18:42 +0100 Subject: [PATCH 026/125] [MINOR][SPARK SUBMIT] Print out R file usage in spark-submit ## What changes were proposed in this pull request? Currently, running the shell below: ```bash $ ./bin/spark-submit tmp.R a b c ``` with R file, `tmp.R` as below: ```r #!/usr/bin/env Rscript library(SparkR) sparkRSQL.init(sparkR.init(master = "local")) collect(createDataFrame(list(list(1)))) print(commandArgs(trailingOnly = TRUE)) ``` working fine as below: ```bash _1 1 1 [1] "a" "b" "c" ``` However, it looks not printed in usage documentation as below: ```bash $ ./bin/spark-submit ``` ``` Usage: spark-submit [options] [app arguments] ... ``` For `./bin/sparkR`, it looks fine as below: ```bash $ ./bin/sparkR tmp.R ``` ``` Running R applications through 'sparkR' is not supported as of Spark 2.0. Use ./bin/spark-submit ``` Running the script below: ```bash $ ./bin/spark-submit ``` **Before** ``` Usage: spark-submit [options] [app arguments] ... ``` **After** ``` Usage: spark-submit [options] [app arguments] ... ``` ## How was this patch tested? Manually tested. Author: hyukjinkwon Closes #18505 from HyukjinKwon/minor-doc-summit. --- .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 3d9a14c51618..7800d3d624e3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -504,7 +504,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S outStream.println("Unknown/unsupported param " + unknownParam) } val command = sys.env.get("_SPARK_CMD_USAGE").getOrElse( - """Usage: spark-submit [options] [app arguments] + """Usage: spark-submit [options] [app arguments] |Usage: spark-submit --kill [submission ID] --master [spark://...] |Usage: spark-submit --status [submission ID] --master [spark://...] |Usage: spark-submit run-example [options] example-class [example args]""".stripMargin) From d492cc5a21cd67b3999b85d97f5c41c3734b1ba3 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Tue, 4 Jul 2017 20:45:58 +0800 Subject: [PATCH 027/125] [SPARK-19507][SPARK-21296][PYTHON] Avoid per-record type dispatch in schema verification and improve exception message ## What changes were proposed in this pull request? **Context** While reviewing https://github.com/apache/spark/pull/17227, I realised here we type-dispatch per record. The PR itself is fine in terms of performance as is but this prints a prefix, `"obj"` in exception message as below: ``` from pyspark.sql.types import * schema = StructType([StructField('s', IntegerType(), nullable=False)]) spark.createDataFrame([["1"]], schema) ... TypeError: obj.s: IntegerType can not accept object '1' in type ``` I suggested to get rid of this but during investigating this, I realised my approach might bring a performance regression as it is a hot path. Only for SPARK-19507 and https://github.com/apache/spark/pull/17227, It needs more changes to cleanly get rid of the prefix and I rather decided to fix both issues together. **Propersal** This PR tried to - get rid of per-record type dispatch as we do in many code paths in Scala so that it improves the performance (roughly ~25% improvement) - SPARK-21296 This was tested with a simple code `spark.createDataFrame(range(1000000), "int")`. However, I am quite sure the actual improvement in practice is larger than this, in particular, when the schema is complicated. - improve error message in exception describing field information as prose - SPARK-19507 ## How was this patch tested? Manually tested and unit tests were added in `python/pyspark/sql/tests.py`. Benchmark - codes: https://gist.github.com/HyukjinKwon/c3397469c56cb26c2d7dd521ed0bc5a3 Error message - codes: https://gist.github.com/HyukjinKwon/b1b2c7f65865444c4a8836435100e398 **Before** Benchmark: - Results: https://gist.github.com/HyukjinKwon/4a291dab45542106301a0c1abcdca924 Error message - Results: https://gist.github.com/HyukjinKwon/57b1916395794ce924faa32b14a3fe19 **After** Benchmark - Results: https://gist.github.com/HyukjinKwon/21496feecc4a920e50c4e455f836266e Error message - Results: https://gist.github.com/HyukjinKwon/7a494e4557fe32a652ce1236e504a395 Closes #17227 Author: hyukjinkwon Author: David Gingrich Closes #18521 from HyukjinKwon/python-type-dispatch. --- python/pyspark/rdd.py | 1 - python/pyspark/sql/session.py | 12 +- python/pyspark/sql/tests.py | 203 ++++++++++++++++++++++++++++++- python/pyspark/sql/types.py | 219 +++++++++++++++++++++++----------- 4 files changed, 352 insertions(+), 83 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 60141792d499..7dfa17f68a94 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -627,7 +627,6 @@ def sortPartition(iterator): def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): """ Sorts this RDD, which is assumed to consist of (key, value) pairs. - # noqa >>> tmp = [('a', 1), ('b', 2), ('1', 3), ('d', 4), ('2', 5)] >>> sc.parallelize(tmp).sortByKey().first() diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index e3bf0f35ea15..2cc0e2d1d7b8 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -33,7 +33,7 @@ from pyspark.sql.dataframe import DataFrame from pyspark.sql.readwriter import DataFrameReader from pyspark.sql.streaming import DataStreamReader -from pyspark.sql.types import Row, DataType, StringType, StructType, _verify_type, \ +from pyspark.sql.types import Row, DataType, StringType, StructType, _make_type_verifier, \ _infer_schema, _has_nulltype, _merge_type, _create_converter, _parse_datatype_string from pyspark.sql.utils import install_exception_handler @@ -514,17 +514,21 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr schema = [str(x) for x in data.columns] data = [r.tolist() for r in data.to_records(index=False)] - verify_func = _verify_type if verifySchema else lambda _, t: True if isinstance(schema, StructType): + verify_func = _make_type_verifier(schema) if verifySchema else lambda _: True + def prepare(obj): - verify_func(obj, schema) + verify_func(obj) return obj elif isinstance(schema, DataType): dataType = schema schema = StructType().add("value", schema) + verify_func = _make_type_verifier( + dataType, name="field value") if verifySchema else lambda _: True + def prepare(obj): - verify_func(obj, dataType) + verify_func(obj) return obj, else: if isinstance(schema, list): diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index c105969b26b9..16ba8bd73f40 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -57,7 +57,7 @@ from pyspark import SparkContext from pyspark.sql import SparkSession, SQLContext, HiveContext, Column, Row from pyspark.sql.types import * -from pyspark.sql.types import UserDefinedType, _infer_type +from pyspark.sql.types import UserDefinedType, _infer_type, _make_type_verifier from pyspark.tests import ReusedPySparkTestCase, SparkSubmitTests from pyspark.sql.functions import UserDefinedFunction, sha2, lit from pyspark.sql.window import Window @@ -852,7 +852,7 @@ def test_convert_row_to_dict(self): self.assertEqual(1.0, row.asDict()['d']['key'].c) def test_udt(self): - from pyspark.sql.types import _parse_datatype_json_string, _infer_type, _verify_type + from pyspark.sql.types import _parse_datatype_json_string, _infer_type, _make_type_verifier from pyspark.sql.tests import ExamplePointUDT, ExamplePoint def check_datatype(datatype): @@ -868,8 +868,8 @@ def check_datatype(datatype): check_datatype(structtype_with_udt) p = ExamplePoint(1.0, 2.0) self.assertEqual(_infer_type(p), ExamplePointUDT()) - _verify_type(ExamplePoint(1.0, 2.0), ExamplePointUDT()) - self.assertRaises(ValueError, lambda: _verify_type([1.0, 2.0], ExamplePointUDT())) + _make_type_verifier(ExamplePointUDT())(ExamplePoint(1.0, 2.0)) + self.assertRaises(ValueError, lambda: _make_type_verifier(ExamplePointUDT())([1.0, 2.0])) check_datatype(PythonOnlyUDT()) structtype_with_udt = StructType([StructField("label", DoubleType(), False), @@ -877,8 +877,10 @@ def check_datatype(datatype): check_datatype(structtype_with_udt) p = PythonOnlyPoint(1.0, 2.0) self.assertEqual(_infer_type(p), PythonOnlyUDT()) - _verify_type(PythonOnlyPoint(1.0, 2.0), PythonOnlyUDT()) - self.assertRaises(ValueError, lambda: _verify_type([1.0, 2.0], PythonOnlyUDT())) + _make_type_verifier(PythonOnlyUDT())(PythonOnlyPoint(1.0, 2.0)) + self.assertRaises( + ValueError, + lambda: _make_type_verifier(PythonOnlyUDT())([1.0, 2.0])) def test_simple_udt_in_df(self): schema = StructType().add("key", LongType()).add("val", PythonOnlyUDT()) @@ -2636,6 +2638,195 @@ def range_frame_match(): importlib.reload(window) + +class DataTypeVerificationTests(unittest.TestCase): + + def test_verify_type_exception_msg(self): + self.assertRaisesRegexp( + ValueError, + "test_name", + lambda: _make_type_verifier(StringType(), nullable=False, name="test_name")(None)) + + schema = StructType([StructField('a', StructType([StructField('b', IntegerType())]))]) + self.assertRaisesRegexp( + TypeError, + "field b in field a", + lambda: _make_type_verifier(schema)([["data"]])) + + def test_verify_type_ok_nullable(self): + obj = None + types = [IntegerType(), FloatType(), StringType(), StructType([])] + for data_type in types: + try: + _make_type_verifier(data_type, nullable=True)(obj) + except Exception: + self.fail("verify_type(%s, %s, nullable=True)" % (obj, data_type)) + + def test_verify_type_not_nullable(self): + import array + import datetime + import decimal + + schema = StructType([ + StructField('s', StringType(), nullable=False), + StructField('i', IntegerType(), nullable=True)]) + + class MyObj: + def __init__(self, **kwargs): + for k, v in kwargs.items(): + setattr(self, k, v) + + # obj, data_type + success_spec = [ + # String + ("", StringType()), + (u"", StringType()), + (1, StringType()), + (1.0, StringType()), + ([], StringType()), + ({}, StringType()), + + # UDT + (ExamplePoint(1.0, 2.0), ExamplePointUDT()), + + # Boolean + (True, BooleanType()), + + # Byte + (-(2**7), ByteType()), + (2**7 - 1, ByteType()), + + # Short + (-(2**15), ShortType()), + (2**15 - 1, ShortType()), + + # Integer + (-(2**31), IntegerType()), + (2**31 - 1, IntegerType()), + + # Long + (2**64, LongType()), + + # Float & Double + (1.0, FloatType()), + (1.0, DoubleType()), + + # Decimal + (decimal.Decimal("1.0"), DecimalType()), + + # Binary + (bytearray([1, 2]), BinaryType()), + + # Date/Timestamp + (datetime.date(2000, 1, 2), DateType()), + (datetime.datetime(2000, 1, 2, 3, 4), DateType()), + (datetime.datetime(2000, 1, 2, 3, 4), TimestampType()), + + # Array + ([], ArrayType(IntegerType())), + (["1", None], ArrayType(StringType(), containsNull=True)), + ([1, 2], ArrayType(IntegerType())), + ((1, 2), ArrayType(IntegerType())), + (array.array('h', [1, 2]), ArrayType(IntegerType())), + + # Map + ({}, MapType(StringType(), IntegerType())), + ({"a": 1}, MapType(StringType(), IntegerType())), + ({"a": None}, MapType(StringType(), IntegerType(), valueContainsNull=True)), + + # Struct + ({"s": "a", "i": 1}, schema), + ({"s": "a", "i": None}, schema), + ({"s": "a"}, schema), + ({"s": "a", "f": 1.0}, schema), + (Row(s="a", i=1), schema), + (Row(s="a", i=None), schema), + (Row(s="a", i=1, f=1.0), schema), + (["a", 1], schema), + (["a", None], schema), + (("a", 1), schema), + (MyObj(s="a", i=1), schema), + (MyObj(s="a", i=None), schema), + (MyObj(s="a"), schema), + ] + + # obj, data_type, exception class + failure_spec = [ + # String (match anything but None) + (None, StringType(), ValueError), + + # UDT + (ExamplePoint(1.0, 2.0), PythonOnlyUDT(), ValueError), + + # Boolean + (1, BooleanType(), TypeError), + ("True", BooleanType(), TypeError), + ([1], BooleanType(), TypeError), + + # Byte + (-(2**7) - 1, ByteType(), ValueError), + (2**7, ByteType(), ValueError), + ("1", ByteType(), TypeError), + (1.0, ByteType(), TypeError), + + # Short + (-(2**15) - 1, ShortType(), ValueError), + (2**15, ShortType(), ValueError), + + # Integer + (-(2**31) - 1, IntegerType(), ValueError), + (2**31, IntegerType(), ValueError), + + # Float & Double + (1, FloatType(), TypeError), + (1, DoubleType(), TypeError), + + # Decimal + (1.0, DecimalType(), TypeError), + (1, DecimalType(), TypeError), + ("1.0", DecimalType(), TypeError), + + # Binary + (1, BinaryType(), TypeError), + + # Date/Timestamp + ("2000-01-02", DateType(), TypeError), + (946811040, TimestampType(), TypeError), + + # Array + (["1", None], ArrayType(StringType(), containsNull=False), ValueError), + ([1, "2"], ArrayType(IntegerType()), TypeError), + + # Map + ({"a": 1}, MapType(IntegerType(), IntegerType()), TypeError), + ({"a": "1"}, MapType(StringType(), IntegerType()), TypeError), + ({"a": None}, MapType(StringType(), IntegerType(), valueContainsNull=False), + ValueError), + + # Struct + ({"s": "a", "i": "1"}, schema, TypeError), + (Row(s="a"), schema, ValueError), # Row can't have missing field + (Row(s="a", i="1"), schema, TypeError), + (["a"], schema, ValueError), + (["a", "1"], schema, TypeError), + (MyObj(s="a", i="1"), schema, TypeError), + (MyObj(s=None, i="1"), schema, ValueError), + ] + + # Check success cases + for obj, data_type in success_spec: + try: + _make_type_verifier(data_type, nullable=False)(obj) + except Exception: + self.fail("verify_type(%s, %s, nullable=False)" % (obj, data_type)) + + # Check failure cases + for obj, data_type, exp in failure_spec: + msg = "verify_type(%s, %s, nullable=False) == %s" % (obj, data_type, exp) + with self.assertRaises(exp, msg=msg): + _make_type_verifier(data_type, nullable=False)(obj) + + if __name__ == "__main__": from pyspark.sql.tests import * if xmlrunner: diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 26b54a7fb370..f5505ed4722a 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1249,121 +1249,196 @@ def _infer_schema_type(obj, dataType): } -def _verify_type(obj, dataType, nullable=True): +def _make_type_verifier(dataType, nullable=True, name=None): """ - Verify the type of obj against dataType, raise a TypeError if they do not match. - - Also verify the value of obj against datatype, raise a ValueError if it's not within the allowed - range, e.g. using 128 as ByteType will overflow. Note that, Python float is not checked, so it - will become infinity when cast to Java float if it overflows. - - >>> _verify_type(None, StructType([])) - >>> _verify_type("", StringType()) - >>> _verify_type(0, LongType()) - >>> _verify_type(list(range(3)), ArrayType(ShortType())) - >>> _verify_type(set(), ArrayType(StringType())) # doctest: +IGNORE_EXCEPTION_DETAIL + Make a verifier that checks the type of obj against dataType and raises a TypeError if they do + not match. + + This verifier also checks the value of obj against datatype and raises a ValueError if it's not + within the allowed range, e.g. using 128 as ByteType will overflow. Note that, Python float is + not checked, so it will become infinity when cast to Java float if it overflows. + + >>> _make_type_verifier(StructType([]))(None) + >>> _make_type_verifier(StringType())("") + >>> _make_type_verifier(LongType())(0) + >>> _make_type_verifier(ArrayType(ShortType()))(list(range(3))) + >>> _make_type_verifier(ArrayType(StringType()))(set()) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... TypeError:... - >>> _verify_type({}, MapType(StringType(), IntegerType())) - >>> _verify_type((), StructType([])) - >>> _verify_type([], StructType([])) - >>> _verify_type([1], StructType([])) # doctest: +IGNORE_EXCEPTION_DETAIL + >>> _make_type_verifier(MapType(StringType(), IntegerType()))({}) + >>> _make_type_verifier(StructType([]))(()) + >>> _make_type_verifier(StructType([]))([]) + >>> _make_type_verifier(StructType([]))([1]) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... ValueError:... >>> # Check if numeric values are within the allowed range. - >>> _verify_type(12, ByteType()) - >>> _verify_type(1234, ByteType()) # doctest: +IGNORE_EXCEPTION_DETAIL + >>> _make_type_verifier(ByteType())(12) + >>> _make_type_verifier(ByteType())(1234) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... ValueError:... - >>> _verify_type(None, ByteType(), False) # doctest: +IGNORE_EXCEPTION_DETAIL + >>> _make_type_verifier(ByteType(), False)(None) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... ValueError:... - >>> _verify_type([1, None], ArrayType(ShortType(), False)) # doctest: +IGNORE_EXCEPTION_DETAIL + >>> _make_type_verifier( + ... ArrayType(ShortType(), False))([1, None]) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... ValueError:... - >>> _verify_type({None: 1}, MapType(StringType(), IntegerType())) + >>> _make_type_verifier(MapType(StringType(), IntegerType()))({None: 1}) Traceback (most recent call last): ... ValueError:... >>> schema = StructType().add("a", IntegerType()).add("b", StringType(), False) - >>> _verify_type((1, None), schema) # doctest: +IGNORE_EXCEPTION_DETAIL + >>> _make_type_verifier(schema)((1, None)) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... ValueError:... """ - if obj is None: - if nullable: - return - else: - raise ValueError("This field is not nullable, but got None") - # StringType can work with any types - if isinstance(dataType, StringType): - return + if name is None: + new_msg = lambda msg: msg + new_name = lambda n: "field %s" % n + else: + new_msg = lambda msg: "%s: %s" % (name, msg) + new_name = lambda n: "field %s in %s" % (n, name) - if isinstance(dataType, UserDefinedType): - if not (hasattr(obj, '__UDT__') and obj.__UDT__ == dataType): - raise ValueError("%r is not an instance of type %r" % (obj, dataType)) - _verify_type(dataType.toInternal(obj), dataType.sqlType()) - return + def verify_nullability(obj): + if obj is None: + if nullable: + return True + else: + raise ValueError(new_msg("This field is not nullable, but got None")) + else: + return False _type = type(dataType) - assert _type in _acceptable_types, "unknown datatype: %s for object %r" % (dataType, obj) - if _type is StructType: - # check the type and fields later - pass - else: + def assert_acceptable_types(obj): + assert _type in _acceptable_types, \ + new_msg("unknown datatype: %s for object %r" % (dataType, obj)) + + def verify_acceptable_types(obj): # subclass of them can not be fromInternal in JVM if type(obj) not in _acceptable_types[_type]: - raise TypeError("%s can not accept object %r in type %s" % (dataType, obj, type(obj))) + raise TypeError(new_msg("%s can not accept object %r in type %s" + % (dataType, obj, type(obj)))) + + if isinstance(dataType, StringType): + # StringType can work with any types + verify_value = lambda _: _ + + elif isinstance(dataType, UserDefinedType): + verifier = _make_type_verifier(dataType.sqlType(), name=name) - if isinstance(dataType, ByteType): - if obj < -128 or obj > 127: - raise ValueError("object of ByteType out of range, got: %s" % obj) + def verify_udf(obj): + if not (hasattr(obj, '__UDT__') and obj.__UDT__ == dataType): + raise ValueError(new_msg("%r is not an instance of type %r" % (obj, dataType))) + verifier(dataType.toInternal(obj)) + + verify_value = verify_udf + + elif isinstance(dataType, ByteType): + def verify_byte(obj): + assert_acceptable_types(obj) + verify_acceptable_types(obj) + if obj < -128 or obj > 127: + raise ValueError(new_msg("object of ByteType out of range, got: %s" % obj)) + + verify_value = verify_byte elif isinstance(dataType, ShortType): - if obj < -32768 or obj > 32767: - raise ValueError("object of ShortType out of range, got: %s" % obj) + def verify_short(obj): + assert_acceptable_types(obj) + verify_acceptable_types(obj) + if obj < -32768 or obj > 32767: + raise ValueError(new_msg("object of ShortType out of range, got: %s" % obj)) + + verify_value = verify_short elif isinstance(dataType, IntegerType): - if obj < -2147483648 or obj > 2147483647: - raise ValueError("object of IntegerType out of range, got: %s" % obj) + def verify_integer(obj): + assert_acceptable_types(obj) + verify_acceptable_types(obj) + if obj < -2147483648 or obj > 2147483647: + raise ValueError( + new_msg("object of IntegerType out of range, got: %s" % obj)) + + verify_value = verify_integer elif isinstance(dataType, ArrayType): - for i in obj: - _verify_type(i, dataType.elementType, dataType.containsNull) + element_verifier = _make_type_verifier( + dataType.elementType, dataType.containsNull, name="element in array %s" % name) + + def verify_array(obj): + assert_acceptable_types(obj) + verify_acceptable_types(obj) + for i in obj: + element_verifier(i) + + verify_value = verify_array elif isinstance(dataType, MapType): - for k, v in obj.items(): - _verify_type(k, dataType.keyType, False) - _verify_type(v, dataType.valueType, dataType.valueContainsNull) + key_verifier = _make_type_verifier(dataType.keyType, False, name="key of map %s" % name) + value_verifier = _make_type_verifier( + dataType.valueType, dataType.valueContainsNull, name="value of map %s" % name) + + def verify_map(obj): + assert_acceptable_types(obj) + verify_acceptable_types(obj) + for k, v in obj.items(): + key_verifier(k) + value_verifier(v) + + verify_value = verify_map elif isinstance(dataType, StructType): - if isinstance(obj, dict): - for f in dataType.fields: - _verify_type(obj.get(f.name), f.dataType, f.nullable) - elif isinstance(obj, Row) and getattr(obj, "__from_dict__", False): - # the order in obj could be different than dataType.fields - for f in dataType.fields: - _verify_type(obj[f.name], f.dataType, f.nullable) - elif isinstance(obj, (tuple, list)): - if len(obj) != len(dataType.fields): - raise ValueError("Length of object (%d) does not match with " - "length of fields (%d)" % (len(obj), len(dataType.fields))) - for v, f in zip(obj, dataType.fields): - _verify_type(v, f.dataType, f.nullable) - elif hasattr(obj, "__dict__"): - d = obj.__dict__ - for f in dataType.fields: - _verify_type(d.get(f.name), f.dataType, f.nullable) - else: - raise TypeError("StructType can not accept object %r in type %s" % (obj, type(obj))) + verifiers = [] + for f in dataType.fields: + verifier = _make_type_verifier(f.dataType, f.nullable, name=new_name(f.name)) + verifiers.append((f.name, verifier)) + + def verify_struct(obj): + assert_acceptable_types(obj) + + if isinstance(obj, dict): + for f, verifier in verifiers: + verifier(obj.get(f)) + elif isinstance(obj, Row) and getattr(obj, "__from_dict__", False): + # the order in obj could be different than dataType.fields + for f, verifier in verifiers: + verifier(obj[f]) + elif isinstance(obj, (tuple, list)): + if len(obj) != len(verifiers): + raise ValueError( + new_msg("Length of object (%d) does not match with " + "length of fields (%d)" % (len(obj), len(verifiers)))) + for v, (_, verifier) in zip(obj, verifiers): + verifier(v) + elif hasattr(obj, "__dict__"): + d = obj.__dict__ + for f, verifier in verifiers: + verifier(d.get(f)) + else: + raise TypeError(new_msg("StructType can not accept object %r in type %s" + % (obj, type(obj)))) + verify_value = verify_struct + + else: + def verify_default(obj): + assert_acceptable_types(obj) + verify_acceptable_types(obj) + + verify_value = verify_default + + def verify(obj): + if not verify_nullability(obj): + verify_value(obj) + + return verify # This is used to unpickle a Row from JVM From 29b1f6b09f98e216af71e893a9da0c4717c80679 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 4 Jul 2017 08:54:07 -0700 Subject: [PATCH 028/125] [SPARK-21256][SQL] Add withSQLConf to Catalyst Test ### What changes were proposed in this pull request? SQLConf is moved to Catalyst. We are adding more and more test cases for verifying the conf-specific behaviors. It is nice to add a helper function to simplify the test cases. ### How was this patch tested? N/A Author: gatorsmile Closes #18469 from gatorsmile/withSQLConf. --- .../InferFiltersFromConstraintsSuite.scala | 5 +-- .../optimizer/OuterJoinEliminationSuite.scala | 6 +--- .../optimizer/PruneFiltersSuite.scala | 6 +--- .../plans/ConstraintPropagationSuite.scala | 24 +++++++------- .../spark/sql/catalyst/plans/PlanTest.scala | 32 ++++++++++++++++++- .../AggregateEstimationSuite.scala | 9 ++---- .../BasicStatsEstimationSuite.scala | 12 +++---- .../spark/sql/SparkSessionBuilderSuite.scala | 3 ++ .../apache/spark/sql/test/SQLTestUtils.scala | 30 ++++------------- 9 files changed, 64 insertions(+), 63 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala index cdc9f25cf877..d2dd469e2d74 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala @@ -206,13 +206,10 @@ class InferFiltersFromConstraintsSuite extends PlanTest { } test("No inferred filter when constraint propagation is disabled") { - try { - SQLConf.get.setConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED, false) + withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "false") { val originalQuery = testRelation.where('a === 1 && 'a === 'b).analyze val optimized = Optimize.execute(originalQuery) comparePlans(optimized, originalQuery) - } finally { - SQLConf.get.unsetConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala index 623ff3d446a5..893c111c2906 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala @@ -234,9 +234,7 @@ class OuterJoinEliminationSuite extends PlanTest { } test("no outer join elimination if constraint propagation is disabled") { - try { - SQLConf.get.setConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED, false) - + withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "false") { val x = testRelation.subquery('x) val y = testRelation1.subquery('y) @@ -251,8 +249,6 @@ class OuterJoinEliminationSuite extends PlanTest { val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, originalQuery.analyze) - } finally { - SQLConf.get.unsetConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala index 706634cdd29b..6d1a05f3c998 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.CONSTRAINT_PROPAGATION_ENABLED class PruneFiltersSuite extends PlanTest { @@ -149,8 +148,7 @@ class PruneFiltersSuite extends PlanTest { ("tr1.a".attr > 10 || "tr1.c".attr < 10) && 'd.attr < 100) - SQLConf.get.setConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED, false) - try { + withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "false") { val optimized = Optimize.execute(queryWithUselessFilter.analyze) // When constraint propagation is disabled, the useless filter won't be pruned. // It gets pushed down. Because the rule `CombineFilters` runs only once, there are redundant @@ -160,8 +158,6 @@ class PruneFiltersSuite extends PlanTest { .join(tr2.where('d.attr < 100).where('d.attr < 100), Inner, Some("tr1.a".attr === "tr2.a".attr)).analyze comparePlans(optimized, correctAnswer) - } finally { - SQLConf.get.unsetConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala index a3948d90b0e4..a37e06d92264 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, DoubleType, IntegerType, LongType, StringType} -class ConstraintPropagationSuite extends SparkFunSuite { +class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { private def resolveColumn(tr: LocalRelation, columnName: String): Expression = resolveColumn(tr.analyze, columnName) @@ -400,26 +400,26 @@ class ConstraintPropagationSuite extends SparkFunSuite { } test("enable/disable constraint propagation") { - try { - val tr = LocalRelation('a.int, 'b.string, 'c.int) - val filterRelation = tr.where('a.attr > 10) + val tr = LocalRelation('a.int, 'b.string, 'c.int) + val filterRelation = tr.where('a.attr > 10) - SQLConf.get.setConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED, true) + withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "true") { assert(filterRelation.analyze.constraints.nonEmpty) + } - SQLConf.get.setConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED, false) + withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "false") { assert(filterRelation.analyze.constraints.isEmpty) + } - val aliasedRelation = tr.where('c.attr > 10 && 'a.attr < 5) - .groupBy('a, 'c, 'b)('a, 'c.as("c1"), count('a).as("a3")).select('c1, 'a, 'a3) + val aliasedRelation = tr.where('c.attr > 10 && 'a.attr < 5) + .groupBy('a, 'c, 'b)('a, 'c.as("c1"), count('a).as("a3")).select('c1, 'a, 'a3) - SQLConf.get.setConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED, true) + withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "true") { assert(aliasedRelation.analyze.constraints.nonEmpty) + } - SQLConf.get.setConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED, false) + withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "false") { assert(aliasedRelation.analyze.constraints.isEmpty) - } finally { - SQLConf.get.unsetConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 6883d23d477e..e9679d336150 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.plans import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression @@ -28,8 +29,9 @@ import org.apache.spark.sql.internal.SQLConf /** * Provides helper methods for comparing plans. */ -abstract class PlanTest extends SparkFunSuite with PredicateHelper { +trait PlanTest extends SparkFunSuite with PredicateHelper { + // TODO(gatorsmile): remove this from PlanTest and all the analyzer/optimizer rules protected val conf = new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true) /** @@ -142,4 +144,32 @@ abstract class PlanTest extends SparkFunSuite with PredicateHelper { plan1 == plan2 } } + + /** + * Sets all SQL configurations specified in `pairs`, calls `f`, and then restore all SQL + * configurations. + */ + protected def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + val conf = SQLConf.get + val (keys, values) = pairs.unzip + val currentValues = keys.map { key => + if (conf.contains(key)) { + Some(conf.getConfString(key)) + } else { + None + } + } + (keys, values).zipped.foreach { (k, v) => + if (SQLConf.staticConfKeys.contains(k)) { + throw new AnalysisException(s"Cannot modify the value of a static config: $k") + } + conf.setConfString(k, v) + } + try f finally { + keys.zip(currentValues).foreach { + case (key, Some(value)) => conf.setConfString(key, value) + case (key, None) => conf.unsetConf(key) + } + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala index 30ddf03bd3c4..23f95a6cc2ac 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala @@ -19,12 +19,13 @@ package org.apache.spark.sql.catalyst.statsEstimation import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, Literal} import org.apache.spark.sql.catalyst.expressions.aggregate.Count +import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils._ import org.apache.spark.sql.internal.SQLConf -class AggregateEstimationSuite extends StatsEstimationTestBase { +class AggregateEstimationSuite extends StatsEstimationTestBase with PlanTest { /** Columns for testing */ private val columnInfo: AttributeMap[ColumnStat] = AttributeMap(Seq( @@ -100,9 +101,7 @@ class AggregateEstimationSuite extends StatsEstimationTestBase { size = Some(4 * (8 + 4)), attributeStats = AttributeMap(Seq("key12").map(nameToColInfo))) - val originalValue = SQLConf.get.getConf(SQLConf.CBO_ENABLED) - try { - SQLConf.get.setConf(SQLConf.CBO_ENABLED, false) + withSQLConf(SQLConf.CBO_ENABLED.key -> "false") { val noGroupAgg = Aggregate(groupingExpressions = Nil, aggregateExpressions = Seq(Alias(Count(Literal(1)), "cnt")()), child) assert(noGroupAgg.stats == @@ -114,8 +113,6 @@ class AggregateEstimationSuite extends StatsEstimationTestBase { assert(hasGroupAgg.stats == // From UnaryNode.computeStats, childSize * outputRowSize / childRowSize Statistics(sizeInBytes = 48 * (8 + 4 + 8) / (8 + 4))) - } finally { - SQLConf.get.setConf(SQLConf.CBO_ENABLED, originalValue) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala index 31a8cbdee977..5fd21a06a109 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.sql.catalyst.statsEstimation import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, Literal} +import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.IntegerType -class BasicStatsEstimationSuite extends StatsEstimationTestBase { +class BasicStatsEstimationSuite extends PlanTest with StatsEstimationTestBase { val attribute = attr("key") val colStat = ColumnStat(distinctCount = 10, min = Some(1), max = Some(10), nullCount = 0, avgLen = 4, maxLen = 4) @@ -82,18 +83,15 @@ class BasicStatsEstimationSuite extends StatsEstimationTestBase { plan: LogicalPlan, expectedStatsCboOn: Statistics, expectedStatsCboOff: Statistics): Unit = { - val originalValue = SQLConf.get.getConf(SQLConf.CBO_ENABLED) - try { + withSQLConf(SQLConf.CBO_ENABLED.key -> "true") { // Invalidate statistics plan.invalidateStatsCache() - SQLConf.get.setConf(SQLConf.CBO_ENABLED, true) assert(plan.stats == expectedStatsCboOn) + } + withSQLConf(SQLConf.CBO_ENABLED.key -> "false") { plan.invalidateStatsCache() - SQLConf.get.setConf(SQLConf.CBO_ENABLED, false) assert(plan.stats == expectedStatsCboOff) - } finally { - SQLConf.get.setConf(SQLConf.CBO_ENABLED, originalValue) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index 4f6d5f79d466..cdac6827082c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.sql.internal.SQLConf /** * Test cases for the builder pattern of [[SparkSession]]. @@ -67,6 +68,8 @@ class SparkSessionBuilderSuite extends SparkFunSuite { assert(activeSession != defaultSession) assert(session == activeSession) assert(session.conf.get("spark-config2") == "a") + assert(session.sessionState.conf == SQLConf.get) + assert(SQLConf.get.getConfString("spark-config2") == "a") SparkSession.clearActiveSession() assert(SparkSession.builder().getOrCreate() == defaultSession) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index d74a7cce25ed..92ee7d596acd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -35,9 +35,11 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.SessionCatalog.DEFAULT_DATABASE import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.FilterExec +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.{UninterruptibleThread, Utils} /** @@ -53,7 +55,8 @@ import org.apache.spark.util.{UninterruptibleThread, Utils} private[sql] trait SQLTestUtils extends SparkFunSuite with Eventually with BeforeAndAfterAll - with SQLTestData { self => + with SQLTestData + with PlanTest { self => protected def sparkContext = spark.sparkContext @@ -89,28 +92,9 @@ private[sql] trait SQLTestUtils } } - /** - * Sets all SQL configurations specified in `pairs`, calls `f`, and then restore all SQL - * configurations. - * - * @todo Probably this method should be moved to a more general place - */ - protected def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { - val (keys, values) = pairs.unzip - val currentValues = keys.map { key => - if (spark.conf.contains(key)) { - Some(spark.conf.get(key)) - } else { - None - } - } - (keys, values).zipped.foreach(spark.conf.set) - try f finally { - keys.zip(currentValues).foreach { - case (key, Some(value)) => spark.conf.set(key, value) - case (key, None) => spark.conf.unset(key) - } - } + protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + SparkSession.setActiveSession(spark) + super.withSQLConf(pairs: _*)(f) } /** From a3c29fcbbda02c1528b4185bcb880c91077d480c Mon Sep 17 00:00:00 2001 From: "YIHAODIAN\\wangshuangshuang" Date: Tue, 4 Jul 2017 09:44:27 -0700 Subject: [PATCH 029/125] [SPARK-19726][SQL] Faild to insert null timestamp value to mysql using spark jdbc ## What changes were proposed in this pull request? when creating table like following: > create table timestamp_test(id int(11), time_stamp timestamp not null default current_timestamp); The result of Excuting "insert into timestamp_test values (111, null)" is different between Spark and JDBC. ``` mysql> select * from timestamp_test; +------+---------------------+ | id | time_stamp | +------+---------------------+ | 111 | 1970-01-01 00:00:00 | -> spark | 111 | 2017-06-27 19:32:38 | -> mysql +------+---------------------+ 2 rows in set (0.00 sec) ``` Because in such case ```StructField.nullable``` is false, so the generated codes of ```InvokeLike``` and ```BoundReference``` don't check whether the field is null or not. Instead, they directly use ```CodegenContext.INPUT_ROW.getLong(1)```, however, ```UnsafeRow.setNullAt(1)``` will put 0 in the underlying memory. The PR will ```always``` set ```StructField.nullable``` true after obtaining metadata from jdbc connection, Since we can insert null to not null timestamp column in MySQL. In this way, spark will propagate null to underlying DB engine, and let DB to choose how to process NULL. ## How was this patch tested? Added tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: YIHAODIAN\wangshuangshuang Author: Shuangshuang Wang Closes #18445 from shuangshuangwang/SPARK-19726. --- .../sql/execution/datasources/jdbc/JDBCRDD.scala | 2 +- .../sql/execution/datasources/jdbc/JdbcUtils.scala | 12 ++++++++++-- .../org/apache/spark/sql/jdbc/JDBCWriteSuite.scala | 8 ++++++++ 3 files changed, 19 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index 0f53b5c7c6f0..57e9bc9b7045 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -59,7 +59,7 @@ object JDBCRDD extends Logging { try { val rs = statement.executeQuery() try { - JdbcUtils.getSchema(rs, dialect) + JdbcUtils.getSchema(rs, dialect, alwaysNullable = true) } finally { rs.close() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index ca61c2efe2dd..55b2539c1338 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -266,10 +266,14 @@ object JdbcUtils extends Logging { /** * Takes a [[ResultSet]] and returns its Catalyst schema. * + * @param alwaysNullable If true, all the columns are nullable. * @return A [[StructType]] giving the Catalyst schema. * @throws SQLException if the schema contains an unsupported type. */ - def getSchema(resultSet: ResultSet, dialect: JdbcDialect): StructType = { + def getSchema( + resultSet: ResultSet, + dialect: JdbcDialect, + alwaysNullable: Boolean = false): StructType = { val rsmd = resultSet.getMetaData val ncols = rsmd.getColumnCount val fields = new Array[StructField](ncols) @@ -290,7 +294,11 @@ object JdbcUtils extends Logging { rsmd.getClass.getName == "org.apache.hive.jdbc.HiveResultSetMetaData" => true } } - val nullable = rsmd.isNullable(i + 1) != ResultSetMetaData.columnNoNulls + val nullable = if (alwaysNullable) { + true + } else { + rsmd.isNullable(i + 1) != ResultSetMetaData.columnNoNulls + } val metadata = new MetadataBuilder() .putString("name", columnName) .putLong("scale", fieldScale) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index bf1fd160704f..92f50a095f19 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters.propertiesAsScalaMapConverter import org.scalatest.BeforeAndAfter +import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} @@ -506,4 +507,11 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { "schema struct")) } } + + test("SPARK-19726: INSERT null to a NOT NULL column") { + val e = intercept[SparkException] { + sql("INSERT INTO PEOPLE1 values (null, null)") + }.getMessage + assert(e.contains("NULL not allowed for column \"NAME\"")) + } } From 1b50e0e0d6fd9d1b815a3bb37647ea659222e3f1 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 4 Jul 2017 09:48:40 -0700 Subject: [PATCH 030/125] [SPARK-20256][SQL] SessionState should be created more lazily ## What changes were proposed in this pull request? `SessionState` is designed to be created lazily. However, in reality, it created immediately in `SparkSession.Builder.getOrCreate` ([here](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala#L943)). This PR aims to recover the lazy behavior by keeping the options into `initialSessionOptions`. The benefit is like the following. Users can start `spark-shell` and use RDD operations without any problems. **BEFORE** ```scala $ bin/spark-shell java.lang.IllegalArgumentException: Error while instantiating 'org.apache.spark.sql.hive.HiveSessionStateBuilder' ... Caused by: org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.security.AccessControlException: Permission denied: user=spark, access=READ, inode="/apps/hive/warehouse":hive:hdfs:drwx------ ``` As reported in SPARK-20256, this happens when the warehouse directory is not allowed for this user. **AFTER** ```scala $ bin/spark-shell ... Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 2.3.0-SNAPSHOT /_/ Using Scala version 2.11.8 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_112) Type in expressions to have them evaluated. Type :help for more information. scala> sc.range(0, 10, 1).count() res0: Long = 10 ``` ## How was this patch tested? Manual. This closes #18512 . Author: Dongjoon Hyun Closes #18501 from dongjoon-hyun/SPARK-20256. --- .../scala/org/apache/spark/sql/SparkSession.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 2c38f7d7c88d..0ddcd2111aa5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -117,6 +117,12 @@ class SparkSession private( existingSharedState.getOrElse(new SharedState(sparkContext)) } + /** + * Initial options for session. This options are applied once when sessionState is created. + */ + @transient + private[sql] val initialSessionOptions = new scala.collection.mutable.HashMap[String, String] + /** * State isolated across sessions, including SQL configurations, temporary tables, registered * functions, and everything else that accepts a [[org.apache.spark.sql.internal.SQLConf]]. @@ -132,9 +138,11 @@ class SparkSession private( parentSessionState .map(_.clone(this)) .getOrElse { - SparkSession.instantiateSessionState( + val state = SparkSession.instantiateSessionState( SparkSession.sessionStateClassName(sparkContext.conf), self) + initialSessionOptions.foreach { case (k, v) => state.conf.setConfString(k, v) } + state } } @@ -940,7 +948,7 @@ object SparkSession { } session = new SparkSession(sparkContext, None, None, extensions) - options.foreach { case (k, v) => session.sessionState.conf.setConfString(k, v) } + options.foreach { case (k, v) => session.initialSessionOptions.put(k, v) } defaultSession.set(session) // Register a successfully instantiated context to the singleton. This should be at the From 4d6d8192c807006ff89488a1d38bc6f7d41de5cf Mon Sep 17 00:00:00 2001 From: dardelet Date: Tue, 4 Jul 2017 17:58:44 +0100 Subject: [PATCH 031/125] [SPARK-21268][MLLIB] Move center calculations to a distributed map in KMeans ## What changes were proposed in this pull request? The scal() and creation of newCenter vector is done in the driver, after a collectAsMap operation while it could be done in the distributed RDD. This PR moves this code before the collectAsMap for more efficiency ## How was this patch tested? This was tested manually by running the KMeansExample and verifying that the new code ran without error and gave same output as before. Author: dardelet Author: Guillaume Dardelet Closes #18491 from dardelet/move-center-calculation-to-distributed-map-kmean. --- .../org/apache/spark/mllib/clustering/KMeans.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) 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 fa72b72e2d92..98e50c5b45cf 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 @@ -272,8 +272,8 @@ class KMeans private ( val costAccum = sc.doubleAccumulator val bcCenters = sc.broadcast(centers) - // Find the sum and count of points mapping to each center - val totalContribs = data.mapPartitions { points => + // Find the new centers + val newCenters = data.mapPartitions { points => val thisCenters = bcCenters.value val dims = thisCenters.head.vector.size @@ -292,15 +292,16 @@ class KMeans private ( }.reduceByKey { case ((sum1, count1), (sum2, count2)) => axpy(1.0, sum2, sum1) (sum1, count1 + count2) + }.mapValues { case (sum, count) => + scal(1.0 / count, sum) + new VectorWithNorm(sum) }.collectAsMap() bcCenters.destroy(blocking = false) // Update the cluster centers and costs converged = true - totalContribs.foreach { case (j, (sum, count)) => - scal(1.0 / count, sum) - val newCenter = new VectorWithNorm(sum) + newCenters.foreach { case (j, newCenter) => if (converged && KMeans.fastSquaredDistance(newCenter, centers(j)) > epsilon * epsilon) { converged = false } From cec392150451a64c9c2902b7f8f4b3b38f25cbea Mon Sep 17 00:00:00 2001 From: actuaryzhang Date: Tue, 4 Jul 2017 12:18:51 -0700 Subject: [PATCH 032/125] [SPARK-20889][SPARKR] Grouped documentation for WINDOW column methods ## What changes were proposed in this pull request? Grouped documentation for column window methods. Author: actuaryzhang Closes #18481 from actuaryzhang/sparkRDocWindow. --- R/pkg/R/functions.R | 225 ++++++++++++++------------------------------ R/pkg/R/generics.R | 28 +++--- 2 files changed, 88 insertions(+), 165 deletions(-) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index a1f5c4f8cc18..8c12308c1d7c 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -200,6 +200,34 @@ NULL #' head(select(tmp, sort_array(tmp$v1, asc = FALSE)))} NULL +#' Window functions for Column operations +#' +#' Window functions defined for \code{Column}. +#' +#' @param x In \code{lag} and \code{lead}, it is the column as a character string or a Column +#' to compute on. In \code{ntile}, it is the number of ntile groups. +#' @param offset In \code{lag}, the number of rows back from the current row from which to obtain +#' a value. In \code{lead}, the number of rows after the current row from which to +#' obtain a value. If not specified, the default is 1. +#' @param defaultValue (optional) default to use when the offset row does not exist. +#' @param ... additional argument(s). +#' @name column_window_functions +#' @rdname column_window_functions +#' @family window functions +#' @examples +#' \dontrun{ +#' # Dataframe used throughout this doc +#' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars)) +#' ws <- orderBy(windowPartitionBy("am"), "hp") +#' tmp <- mutate(df, dist = over(cume_dist(), ws), dense_rank = over(dense_rank(), ws), +#' lag = over(lag(df$mpg), ws), lead = over(lead(df$mpg, 1), ws), +#' percent_rank = over(percent_rank(), ws), +#' rank = over(rank(), ws), row_number = over(row_number(), ws)) +#' # Get ntile group id (1-4) for hp +#' tmp <- mutate(tmp, ntile = over(ntile(4), ws)) +#' head(tmp)} +NULL + #' @details #' \code{lit}: A new Column is created to represent the literal value. #' If the parameter is a Column, it is returned unchanged. @@ -2844,27 +2872,16 @@ setMethod("ifelse", ###################### Window functions###################### -#' cume_dist -#' -#' Window function: returns the cumulative distribution of values within a window partition, -#' i.e. the fraction of rows that are below the current row. -#' -#' N = total number of rows in the partition -#' cume_dist(x) = number of values before (and including) x / N -#' +#' @details +#' \code{cume_dist}: Returns the cumulative distribution of values within a window partition, +#' i.e. the fraction of rows that are below the current row: +#' (number of values before and including x) / (total number of rows in the partition). #' This is equivalent to the \code{CUME_DIST} function in SQL. +#' The method should be used with no argument. #' -#' @rdname cume_dist -#' @name cume_dist -#' @family window functions -#' @aliases cume_dist,missing-method +#' @rdname column_window_functions +#' @aliases cume_dist cume_dist,missing-method #' @export -#' @examples -#' \dontrun{ -#' df <- createDataFrame(mtcars) -#' ws <- orderBy(windowPartitionBy("am"), "hp") -#' out <- select(df, over(cume_dist(), ws), df$hp, df$am) -#' } #' @note cume_dist since 1.6.0 setMethod("cume_dist", signature("missing"), @@ -2873,28 +2890,19 @@ setMethod("cume_dist", column(jc) }) -#' dense_rank -#' -#' Window function: returns the rank of rows within a window partition, without any gaps. +#' @details +#' \code{dense_rank}: Returns the rank of rows within a window partition, without any gaps. #' The difference between rank and dense_rank is that dense_rank leaves no gaps in ranking #' sequence when there are ties. That is, if you were ranking a competition using dense_rank #' and had three people tie for second place, you would say that all three were in second #' place and that the next person came in third. Rank would give me sequential numbers, making #' the person that came in third place (after the ties) would register as coming in fifth. -#' #' This is equivalent to the \code{DENSE_RANK} function in SQL. +#' The method should be used with no argument. #' -#' @rdname dense_rank -#' @name dense_rank -#' @family window functions -#' @aliases dense_rank,missing-method +#' @rdname column_window_functions +#' @aliases dense_rank dense_rank,missing-method #' @export -#' @examples -#' \dontrun{ -#' df <- createDataFrame(mtcars) -#' ws <- orderBy(windowPartitionBy("am"), "hp") -#' out <- select(df, over(dense_rank(), ws), df$hp, df$am) -#' } #' @note dense_rank since 1.6.0 setMethod("dense_rank", signature("missing"), @@ -2903,34 +2911,15 @@ setMethod("dense_rank", column(jc) }) -#' lag -#' -#' Window function: returns the value that is \code{offset} rows before the current row, and +#' @details +#' \code{lag}: Returns the value that is \code{offset} rows before the current row, and #' \code{defaultValue} if there is less than \code{offset} rows before the current row. For example, #' an \code{offset} of one will return the previous row at any given point in the window partition. -#' #' This is equivalent to the \code{LAG} function in SQL. #' -#' @param x the column as a character string or a Column to compute on. -#' @param offset the number of rows back from the current row from which to obtain a value. -#' If not specified, the default is 1. -#' @param defaultValue (optional) default to use when the offset row does not exist. -#' @param ... further arguments to be passed to or from other methods. -#' @rdname lag -#' @name lag -#' @aliases lag,characterOrColumn-method -#' @family window functions +#' @rdname column_window_functions +#' @aliases lag lag,characterOrColumn-method #' @export -#' @examples -#' \dontrun{ -#' df <- createDataFrame(mtcars) -#' -#' # Partition by am (transmission) and order by hp (horsepower) -#' ws <- orderBy(windowPartitionBy("am"), "hp") -#' -#' # Lag mpg values by 1 row on the partition-and-ordered table -#' out <- select(df, over(lag(df$mpg), ws), df$mpg, df$hp, df$am) -#' } #' @note lag since 1.6.0 setMethod("lag", signature(x = "characterOrColumn"), @@ -2946,35 +2935,16 @@ setMethod("lag", column(jc) }) -#' lead -#' -#' Window function: returns the value that is \code{offset} rows after the current row, and +#' @details +#' \code{lead}: Returns the value that is \code{offset} rows after the current row, and #' \code{defaultValue} if there is less than \code{offset} rows after the current row. #' For example, an \code{offset} of one will return the next row at any given point #' in the window partition. -#' #' This is equivalent to the \code{LEAD} function in SQL. #' -#' @param x the column as a character string or a Column to compute on. -#' @param offset the number of rows after the current row from which to obtain a value. -#' If not specified, the default is 1. -#' @param defaultValue (optional) default to use when the offset row does not exist. -#' -#' @rdname lead -#' @name lead -#' @family window functions -#' @aliases lead,characterOrColumn,numeric-method +#' @rdname column_window_functions +#' @aliases lead lead,characterOrColumn,numeric-method #' @export -#' @examples -#' \dontrun{ -#' df <- createDataFrame(mtcars) -#' -#' # Partition by am (transmission) and order by hp (horsepower) -#' ws <- orderBy(windowPartitionBy("am"), "hp") -#' -#' # Lead mpg values by 1 row on the partition-and-ordered table -#' out <- select(df, over(lead(df$mpg), ws), df$mpg, df$hp, df$am) -#' } #' @note lead since 1.6.0 setMethod("lead", signature(x = "characterOrColumn", offset = "numeric", defaultValue = "ANY"), @@ -2990,31 +2960,15 @@ setMethod("lead", column(jc) }) -#' ntile -#' -#' Window function: returns the ntile group id (from 1 to n inclusive) in an ordered window +#' @details +#' \code{ntile}: Returns the ntile group id (from 1 to n inclusive) in an ordered window #' partition. For example, if n is 4, the first quarter of the rows will get value 1, the second #' quarter will get 2, the third quarter will get 3, and the last quarter will get 4. -#' #' This is equivalent to the \code{NTILE} function in SQL. #' -#' @param x Number of ntile groups -#' -#' @rdname ntile -#' @name ntile -#' @aliases ntile,numeric-method -#' @family window functions +#' @rdname column_window_functions +#' @aliases ntile ntile,numeric-method #' @export -#' @examples -#' \dontrun{ -#' df <- createDataFrame(mtcars) -#' -#' # Partition by am (transmission) and order by hp (horsepower) -#' ws <- orderBy(windowPartitionBy("am"), "hp") -#' -#' # Get ntile group id (1-4) for hp -#' out <- select(df, over(ntile(4), ws), df$hp, df$am) -#' } #' @note ntile since 1.6.0 setMethod("ntile", signature(x = "numeric"), @@ -3023,27 +2977,15 @@ setMethod("ntile", column(jc) }) -#' percent_rank -#' -#' Window function: returns the relative rank (i.e. percentile) of rows within a window partition. -#' -#' This is computed by: -#' -#' (rank of row in its partition - 1) / (number of rows in the partition - 1) -#' -#' This is equivalent to the PERCENT_RANK function in SQL. +#' @details +#' \code{percent_rank}: Returns the relative rank (i.e. percentile) of rows within a window partition. +#' This is computed by: (rank of row in its partition - 1) / (number of rows in the partition - 1). +#' This is equivalent to the \code{PERCENT_RANK} function in SQL. +#' The method should be used with no argument. #' -#' @rdname percent_rank -#' @name percent_rank -#' @family window functions -#' @aliases percent_rank,missing-method +#' @rdname column_window_functions +#' @aliases percent_rank percent_rank,missing-method #' @export -#' @examples -#' \dontrun{ -#' df <- createDataFrame(mtcars) -#' ws <- orderBy(windowPartitionBy("am"), "hp") -#' out <- select(df, over(percent_rank(), ws), df$hp, df$am) -#' } #' @note percent_rank since 1.6.0 setMethod("percent_rank", signature("missing"), @@ -3052,29 +2994,19 @@ setMethod("percent_rank", column(jc) }) -#' rank -#' -#' Window function: returns the rank of rows within a window partition. -#' +#' @details +#' \code{rank}: Returns the rank of rows within a window partition. #' The difference between rank and dense_rank is that dense_rank leaves no gaps in ranking #' sequence when there are ties. That is, if you were ranking a competition using dense_rank #' and had three people tie for second place, you would say that all three were in second #' place and that the next person came in third. Rank would give me sequential numbers, making #' the person that came in third place (after the ties) would register as coming in fifth. +#' This is equivalent to the \code{RANK} function in SQL. +#' The method should be used with no argument. #' -#' This is equivalent to the RANK function in SQL. -#' -#' @rdname rank -#' @name rank -#' @family window functions -#' @aliases rank,missing-method +#' @rdname column_window_functions +#' @aliases rank rank,missing-method #' @export -#' @examples -#' \dontrun{ -#' df <- createDataFrame(mtcars) -#' ws <- orderBy(windowPartitionBy("am"), "hp") -#' out <- select(df, over(rank(), ws), df$hp, df$am) -#' } #' @note rank since 1.6.0 setMethod("rank", signature(x = "missing"), @@ -3083,11 +3015,7 @@ setMethod("rank", column(jc) }) -# Expose rank() in the R base package -#' @param x a numeric, complex, character or logical vector. -#' @param ... additional argument(s) passed to the method. -#' @name rank -#' @rdname rank +#' @rdname column_window_functions #' @aliases rank,ANY-method #' @export setMethod("rank", @@ -3096,23 +3024,14 @@ setMethod("rank", base::rank(x, ...) }) -#' row_number -#' -#' Window function: returns a sequential number starting at 1 within a window partition. -#' -#' This is equivalent to the ROW_NUMBER function in SQL. +#' @details +#' \code{row_number}: Returns a sequential number starting at 1 within a window partition. +#' This is equivalent to the \code{ROW_NUMBER} function in SQL. +#' The method should be used with no argument. #' -#' @rdname row_number -#' @name row_number -#' @aliases row_number,missing-method -#' @family window functions +#' @rdname column_window_functions +#' @aliases row_number row_number,missing-method #' @export -#' @examples -#' \dontrun{ -#' df <- createDataFrame(mtcars) -#' ws <- orderBy(windowPartitionBy("am"), "hp") -#' out <- select(df, over(row_number(), ws), df$hp, df$am) -#' } #' @note row_number since 1.6.0 setMethod("row_number", signature("missing"), diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index b901b74e4728..beac18e41273 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1013,9 +1013,9 @@ setGeneric("create_map", function(x, ...) { standardGeneric("create_map") }) #' @name NULL setGeneric("hash", function(x, ...) { standardGeneric("hash") }) -#' @param x empty. Should be used with no argument. -#' @rdname cume_dist +#' @rdname column_window_functions #' @export +#' @name NULL setGeneric("cume_dist", function(x = "missing") { standardGeneric("cume_dist") }) #' @rdname column_datetime_diff_functions @@ -1053,9 +1053,9 @@ setGeneric("dayofyear", function(x) { standardGeneric("dayofyear") }) #' @name NULL setGeneric("decode", function(x, charset) { standardGeneric("decode") }) -#' @param x empty. Should be used with no argument. -#' @rdname dense_rank +#' @rdname column_window_functions #' @export +#' @name NULL setGeneric("dense_rank", function(x = "missing") { standardGeneric("dense_rank") }) #' @rdname column_string_functions @@ -1159,8 +1159,9 @@ setGeneric("isnan", function(x) { standardGeneric("isnan") }) #' @name NULL setGeneric("kurtosis", function(x) { standardGeneric("kurtosis") }) -#' @rdname lag +#' @rdname column_window_functions #' @export +#' @name NULL setGeneric("lag", function(x, ...) { standardGeneric("lag") }) #' @rdname last @@ -1172,8 +1173,9 @@ setGeneric("last", function(x, ...) { standardGeneric("last") }) #' @name NULL setGeneric("last_day", function(x) { standardGeneric("last_day") }) -#' @rdname lead +#' @rdname column_window_functions #' @export +#' @name NULL setGeneric("lead", function(x, offset, defaultValue = NULL) { standardGeneric("lead") }) #' @rdname column_nonaggregate_functions @@ -1260,8 +1262,9 @@ setGeneric("not", function(x) { standardGeneric("not") }) #' @name NULL setGeneric("next_day", function(y, x) { standardGeneric("next_day") }) -#' @rdname ntile +#' @rdname column_window_functions #' @export +#' @name NULL setGeneric("ntile", function(x) { standardGeneric("ntile") }) #' @rdname column_aggregate_functions @@ -1269,9 +1272,9 @@ setGeneric("ntile", function(x) { standardGeneric("ntile") }) #' @name NULL setGeneric("n_distinct", function(x, ...) { standardGeneric("n_distinct") }) -#' @param x empty. Should be used with no argument. -#' @rdname percent_rank +#' @rdname column_window_functions #' @export +#' @name NULL setGeneric("percent_rank", function(x = "missing") { standardGeneric("percent_rank") }) #' @rdname column_math_functions @@ -1304,8 +1307,9 @@ setGeneric("rand", function(seed) { standardGeneric("rand") }) #' @name NULL setGeneric("randn", function(seed) { standardGeneric("randn") }) -#' @rdname rank +#' @rdname column_window_functions #' @export +#' @name NULL setGeneric("rank", function(x, ...) { standardGeneric("rank") }) #' @rdname column_string_functions @@ -1334,9 +1338,9 @@ setGeneric("reverse", function(x) { standardGeneric("reverse") }) #' @name NULL setGeneric("rint", function(x) { standardGeneric("rint") }) -#' @param x empty. Should be used with no argument. -#' @rdname row_number +#' @rdname column_window_functions #' @export +#' @name NULL setGeneric("row_number", function(x = "missing") { standardGeneric("row_number") }) #' @rdname column_string_functions From daabf425ec0272951b11f286e4bec7a48f42cc0d Mon Sep 17 00:00:00 2001 From: wangmiao1981 Date: Tue, 4 Jul 2017 12:37:29 -0700 Subject: [PATCH 033/125] [MINOR][SPARKR] ignore Rplots.pdf test output after running R tests ## What changes were proposed in this pull request? After running R tests in local build, it outputs Rplots.pdf. This one should be ignored in the git repository. Author: wangmiao1981 Closes #18518 from wangmiao1981/ignore. --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 1d91b43c23fa..cf9780db37ad 100644 --- a/.gitignore +++ b/.gitignore @@ -25,6 +25,7 @@ R-unit-tests.log R/unit-tests.out R/cran-check.out R/pkg/vignettes/sparkr-vignettes.html +R/pkg/tests/fulltests/Rplots.pdf build/*.jar build/apache-maven* build/scala* From de14086e1f6a2474bb9ba1452ada94e0ce58cf9c Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 5 Jul 2017 10:40:02 +0800 Subject: [PATCH 034/125] [SPARK-21295][SQL] Use qualified names in error message for missing references ### What changes were proposed in this pull request? It is strange to see the following error message. Actually, the column is from another table. ``` cannot resolve '`right.a`' given input columns: [a, c, d]; ``` After the PR, the error message looks like ``` cannot resolve '`right.a`' given input columns: [left.a, right.c, right.d]; ``` ### How was this patch tested? Added a test case Author: gatorsmile Closes #18520 from gatorsmile/removeSQLConf. --- .../sql/catalyst/analysis/CheckAnalysis.scala | 2 +- .../results/columnresolution-negative.sql.out | 10 +++++----- .../results/columnresolution-views.sql.out | 2 +- .../sql-tests/results/columnresolution.sql.out | 18 +++++++++--------- .../sql-tests/results/group-by.sql.out | 2 +- .../sql-tests/results/table-aliases.sql.out | 2 +- .../org/apache/spark/sql/SubquerySuite.scala | 4 ++-- 7 files changed, 20 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index fb81a7006bc5..85c52792ef65 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -86,7 +86,7 @@ trait CheckAnalysis extends PredicateHelper { case operator: LogicalPlan => operator transformExpressionsUp { case a: Attribute if !a.resolved => - val from = operator.inputSet.map(_.name).mkString(", ") + val from = operator.inputSet.map(_.qualifiedName).mkString(", ") a.failAnalysis(s"cannot resolve '${a.sql}' given input columns: [$from]") case e: Expression if e.checkInputDataTypes().isFailure => diff --git a/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out b/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out index 60bd8e9cc99d..9e60e592c2bd 100644 --- a/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out @@ -90,7 +90,7 @@ SELECT mydb1.t1.i1 FROM t1, mydb1.t1 struct<> -- !query 10 output org.apache.spark.sql.AnalysisException -cannot resolve '`mydb1.t1.i1`' given input columns: [i1, i1]; line 1 pos 7 +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1, t1.i1]; line 1 pos 7 -- !query 11 @@ -161,7 +161,7 @@ SELECT db1.t1.i1 FROM t1, mydb2.t1 struct<> -- !query 18 output org.apache.spark.sql.AnalysisException -cannot resolve '`db1.t1.i1`' given input columns: [i1, i1]; line 1 pos 7 +cannot resolve '`db1.t1.i1`' given input columns: [t1.i1, t1.i1]; line 1 pos 7 -- !query 19 @@ -186,7 +186,7 @@ SELECT mydb1.t1 FROM t1 struct<> -- !query 21 output org.apache.spark.sql.AnalysisException -cannot resolve '`mydb1.t1`' given input columns: [i1]; line 1 pos 7 +cannot resolve '`mydb1.t1`' given input columns: [t1.i1]; line 1 pos 7 -- !query 22 @@ -204,7 +204,7 @@ SELECT t1 FROM mydb1.t1 struct<> -- !query 23 output org.apache.spark.sql.AnalysisException -cannot resolve '`t1`' given input columns: [i1]; line 1 pos 7 +cannot resolve '`t1`' given input columns: [t1.i1]; line 1 pos 7 -- !query 24 @@ -221,7 +221,7 @@ SELECT mydb1.t1.i1 FROM t1 struct<> -- !query 25 output org.apache.spark.sql.AnalysisException -cannot resolve '`mydb1.t1.i1`' given input columns: [i1]; line 1 pos 7 +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1]; line 1 pos 7 -- !query 26 diff --git a/sql/core/src/test/resources/sql-tests/results/columnresolution-views.sql.out b/sql/core/src/test/resources/sql-tests/results/columnresolution-views.sql.out index 616421d6f2b2..7c451c2aa5b5 100644 --- a/sql/core/src/test/resources/sql-tests/results/columnresolution-views.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/columnresolution-views.sql.out @@ -105,7 +105,7 @@ SELECT global_temp.view1.i1 FROM global_temp.view1 struct<> -- !query 12 output org.apache.spark.sql.AnalysisException -cannot resolve '`global_temp.view1.i1`' given input columns: [i1]; line 1 pos 7 +cannot resolve '`global_temp.view1.i1`' given input columns: [view1.i1]; line 1 pos 7 -- !query 13 diff --git a/sql/core/src/test/resources/sql-tests/results/columnresolution.sql.out b/sql/core/src/test/resources/sql-tests/results/columnresolution.sql.out index 764cad0e3943..d3ca4443cce5 100644 --- a/sql/core/src/test/resources/sql-tests/results/columnresolution.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/columnresolution.sql.out @@ -96,7 +96,7 @@ SELECT mydb1.t1.i1 FROM t1 struct<> -- !query 11 output org.apache.spark.sql.AnalysisException -cannot resolve '`mydb1.t1.i1`' given input columns: [i1]; line 1 pos 7 +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1]; line 1 pos 7 -- !query 12 @@ -105,7 +105,7 @@ SELECT mydb1.t1.i1 FROM mydb1.t1 struct<> -- !query 12 output org.apache.spark.sql.AnalysisException -cannot resolve '`mydb1.t1.i1`' given input columns: [i1]; line 1 pos 7 +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1]; line 1 pos 7 -- !query 13 @@ -154,7 +154,7 @@ SELECT mydb1.t1.i1 FROM mydb1.t1 struct<> -- !query 18 output org.apache.spark.sql.AnalysisException -cannot resolve '`mydb1.t1.i1`' given input columns: [i1]; line 1 pos 7 +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1]; line 1 pos 7 -- !query 19 @@ -270,7 +270,7 @@ SELECT * FROM mydb1.t3 WHERE c1 IN struct<> -- !query 32 output org.apache.spark.sql.AnalysisException -cannot resolve '`mydb1.t4.c3`' given input columns: [c2, c3]; line 2 pos 42 +cannot resolve '`mydb1.t4.c3`' given input columns: [t4.c2, t4.c3]; line 2 pos 42 -- !query 33 @@ -287,7 +287,7 @@ SELECT mydb1.t1.i1 FROM t1, mydb2.t1 struct<> -- !query 34 output org.apache.spark.sql.AnalysisException -cannot resolve '`mydb1.t1.i1`' given input columns: [i1, i1]; line 1 pos 7 +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1, t1.i1]; line 1 pos 7 -- !query 35 @@ -296,7 +296,7 @@ SELECT mydb1.t1.i1 FROM mydb1.t1, mydb2.t1 struct<> -- !query 35 output org.apache.spark.sql.AnalysisException -cannot resolve '`mydb1.t1.i1`' given input columns: [i1, i1]; line 1 pos 7 +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1, t1.i1]; line 1 pos 7 -- !query 36 @@ -313,7 +313,7 @@ SELECT mydb1.t1.i1 FROM t1, mydb1.t1 struct<> -- !query 37 output org.apache.spark.sql.AnalysisException -cannot resolve '`mydb1.t1.i1`' given input columns: [i1, i1]; line 1 pos 7 +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1, t1.i1]; line 1 pos 7 -- !query 38 @@ -402,7 +402,7 @@ SELECT mydb1.t5.t5.i1 FROM mydb1.t5 struct<> -- !query 48 output org.apache.spark.sql.AnalysisException -cannot resolve '`mydb1.t5.t5.i1`' given input columns: [i1, t5]; line 1 pos 7 +cannot resolve '`mydb1.t5.t5.i1`' given input columns: [t5.i1, t5.t5]; line 1 pos 7 -- !query 49 @@ -411,7 +411,7 @@ SELECT mydb1.t5.t5.i2 FROM mydb1.t5 struct<> -- !query 49 output org.apache.spark.sql.AnalysisException -cannot resolve '`mydb1.t5.t5.i2`' given input columns: [i1, t5]; line 1 pos 7 +cannot resolve '`mydb1.t5.t5.i2`' given input columns: [t5.i1, t5.t5]; line 1 pos 7 -- !query 50 diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 14679850c692..e23ebd4e822f 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -202,7 +202,7 @@ SELECT a AS k, COUNT(b) FROM testData GROUP BY k struct<> -- !query 21 output org.apache.spark.sql.AnalysisException -cannot resolve '`k`' given input columns: [a, b]; line 1 pos 47 +cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 47 -- !query 22 diff --git a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out index c318018dced2..7abbcd834a52 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out @@ -60,4 +60,4 @@ SELECT a AS col1, b AS col2 FROM testData AS t(c, d) struct<> -- !query 6 output org.apache.spark.sql.AnalysisException -cannot resolve '`a`' given input columns: [c, d]; line 1 pos 7 +cannot resolve '`a`' given input columns: [t.c, t.d]; line 1 pos 7 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 820cff655c4f..c0a3b5add313 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -870,9 +870,9 @@ class SubquerySuite extends QueryTest with SharedSQLContext { test("SPARK-20688: correctly check analysis for scalar sub-queries") { withTempView("t") { - Seq(1 -> "a").toDF("i", "j").createTempView("t") + Seq(1 -> "a").toDF("i", "j").createOrReplaceTempView("t") val e = intercept[AnalysisException](sql("SELECT (SELECT count(*) FROM t WHERE a = 1)")) - assert(e.message.contains("cannot resolve '`a`' given input columns: [i, j]")) + assert(e.message.contains("cannot resolve '`a`' given input columns: [t.i, t.j]")) } } } From ce10545d3401c555e56a214b7c2f334274803660 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Wed, 5 Jul 2017 11:24:38 +0800 Subject: [PATCH 035/125] [SPARK-21300][SQL] ExternalMapToCatalyst should null-check map key prior to converting to internal value. ## What changes were proposed in this pull request? `ExternalMapToCatalyst` should null-check map key prior to converting to internal value to throw an appropriate Exception instead of something like NPE. ## How was this patch tested? Added a test and existing tests. Author: Takuya UESHIN Closes #18524 from ueshin/issues/SPARK-21300. --- .../spark/sql/catalyst/JavaTypeInference.scala | 1 + .../spark/sql/catalyst/ScalaReflection.scala | 1 + .../catalyst/expressions/objects/objects.scala | 16 +++++++++++++++- .../encoders/ExpressionEncoderSuite.scala | 8 +++++++- 4 files changed, 24 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index 7683ee7074e7..90ec699877de 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -418,6 +418,7 @@ object JavaTypeInference { inputObject, ObjectType(keyType.getRawType), serializerFor(_, keyType), + keyNullable = true, ObjectType(valueType.getRawType), serializerFor(_, valueType), valueNullable = true 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 d580cf4d3391..f3c1e4150017 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 @@ -494,6 +494,7 @@ object ScalaReflection extends ScalaReflection { inputObject, dataTypeFor(keyType), serializerFor(_, keyType, keyPath, seenTypeSet), + keyNullable = !keyType.typeSymbol.asClass.isPrimitive, dataTypeFor(valueType), serializerFor(_, valueType, valuePath, seenTypeSet), valueNullable = !valueType.typeSymbol.asClass.isPrimitive) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 4b651836ff4d..d6d06aecc077 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -841,18 +841,21 @@ object ExternalMapToCatalyst { inputMap: Expression, keyType: DataType, keyConverter: Expression => Expression, + keyNullable: Boolean, valueType: DataType, valueConverter: Expression => Expression, valueNullable: Boolean): ExternalMapToCatalyst = { val id = curId.getAndIncrement() val keyName = "ExternalMapToCatalyst_key" + id + val keyIsNull = "ExternalMapToCatalyst_key_isNull" + id val valueName = "ExternalMapToCatalyst_value" + id val valueIsNull = "ExternalMapToCatalyst_value_isNull" + id ExternalMapToCatalyst( keyName, + keyIsNull, keyType, - keyConverter(LambdaVariable(keyName, "false", keyType, false)), + keyConverter(LambdaVariable(keyName, keyIsNull, keyType, keyNullable)), valueName, valueIsNull, valueType, @@ -868,6 +871,8 @@ object ExternalMapToCatalyst { * * @param key the name of the map key variable that used when iterate the map, and used as input for * the `keyConverter` + * @param keyIsNull the nullability of the map key variable that used when iterate the map, and + * used as input for the `keyConverter` * @param keyType the data type of the map key variable that used when iterate the map, and used as * input for the `keyConverter` * @param keyConverter A function that take the `key` as input, and converts it to catalyst format. @@ -883,6 +888,7 @@ object ExternalMapToCatalyst { */ case class ExternalMapToCatalyst private( key: String, + keyIsNull: String, keyType: DataType, keyConverter: Expression, value: String, @@ -913,6 +919,7 @@ case class ExternalMapToCatalyst private( val keyElementJavaType = ctx.javaType(keyType) val valueElementJavaType = ctx.javaType(valueType) + ctx.addMutableState("boolean", keyIsNull, "") ctx.addMutableState(keyElementJavaType, key, "") ctx.addMutableState("boolean", valueIsNull, "") ctx.addMutableState(valueElementJavaType, value, "") @@ -950,6 +957,12 @@ case class ExternalMapToCatalyst private( defineEntries -> defineKeyValue } + val keyNullCheck = if (ctx.isPrimitiveType(keyType)) { + s"$keyIsNull = false;" + } else { + s"$keyIsNull = $key == null;" + } + val valueNullCheck = if (ctx.isPrimitiveType(valueType)) { s"$valueIsNull = false;" } else { @@ -972,6 +985,7 @@ case class ExternalMapToCatalyst private( $defineEntries while($entries.hasNext()) { $defineKeyValue + $keyNullCheck $valueNullCheck ${genKeyConverter.code} 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 080f11b76938..bb1955a1ae24 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 @@ -355,12 +355,18 @@ class ExpressionEncoderSuite extends PlanTest with AnalysisTest { checkNullable[String](true) } - test("null check for map key") { + test("null check for map key: String") { val encoder = ExpressionEncoder[Map[String, Int]]() val e = intercept[RuntimeException](encoder.toRow(Map(("a", 1), (null, 2)))) assert(e.getMessage.contains("Cannot use null as map key")) } + test("null check for map key: Integer") { + val encoder = ExpressionEncoder[Map[Integer, String]]() + val e = intercept[RuntimeException](encoder.toRow(Map((1, "a"), (null, "b")))) + assert(e.getMessage.contains("Cannot use null as map key")) + } + private def encodeDecodeTest[T : ExpressionEncoder]( input: T, testName: String): Unit = { From e9a93f8140c913b91781b35e0e1b051c30244882 Mon Sep 17 00:00:00 2001 From: actuaryzhang Date: Tue, 4 Jul 2017 21:05:05 -0700 Subject: [PATCH 036/125] [SPARK-20889][SPARKR][FOLLOWUP] Clean up grouped doc for column methods ## What changes were proposed in this pull request? Add doc for methods that were left out, and fix various style and consistency issues. Author: actuaryzhang Closes #18493 from actuaryzhang/sparkRDocCleanup. --- R/pkg/R/functions.R | 100 ++++++++++++++++++++------------------------ R/pkg/R/generics.R | 7 ++-- 2 files changed, 49 insertions(+), 58 deletions(-) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 8c12308c1d7c..c529d83060f5 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -38,10 +38,10 @@ NULL #' #' Date time functions defined for \code{Column}. #' -#' @param x Column to compute on. +#' @param x Column to compute on. In \code{window}, it must be a time Column of \code{TimestampType}. #' @param format For \code{to_date} and \code{to_timestamp}, it is the string to use to parse -#' x Column to DateType or TimestampType. For \code{trunc}, it is the string used -#' for specifying the truncation method. For example, "year", "yyyy", "yy" for +#' Column \code{x} to DateType or TimestampType. For \code{trunc}, it is the string +#' to use to specify the truncation method. For example, "year", "yyyy", "yy" for #' truncate by year, or "month", "mon", "mm" for truncate by month. #' @param ... additional argument(s). #' @name column_datetime_functions @@ -122,7 +122,7 @@ NULL #' format to. See 'Details'. #' } #' @param y Column to compute on. -#' @param ... additional columns. +#' @param ... additional Columns. #' @name column_string_functions #' @rdname column_string_functions #' @family string functions @@ -167,8 +167,7 @@ NULL #' tmp <- mutate(df, v1 = crc32(df$model), v2 = hash(df$model), #' v3 = hash(df$model, df$mpg), v4 = md5(df$model), #' v5 = sha1(df$model), v6 = sha2(df$model, 256)) -#' head(tmp) -#' } +#' head(tmp)} NULL #' Collection functions for Column operations @@ -190,7 +189,6 @@ NULL #' \dontrun{ #' # Dataframe used throughout this doc #' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars)) -#' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars)) #' tmp <- mutate(df, v1 = create_array(df$mpg, df$cyl, df$hp)) #' head(select(tmp, array_contains(tmp$v1, 21), size(tmp$v1))) #' tmp2 <- mutate(tmp, v2 = explode(tmp$v1)) @@ -394,7 +392,7 @@ setMethod("base64", }) #' @details -#' \code{bin}: An expression that returns the string representation of the binary value +#' \code{bin}: Returns the string representation of the binary value #' of the given long column. For example, bin("12") returns "1100". #' #' @rdname column_math_functions @@ -722,7 +720,7 @@ setMethod("dayofyear", #' \code{decode}: Computes the first argument into a string from a binary using the provided #' character set. #' -#' @param charset Character set to use (one of "US-ASCII", "ISO-8859-1", "UTF-8", "UTF-16BE", +#' @param charset character set to use (one of "US-ASCII", "ISO-8859-1", "UTF-8", "UTF-16BE", #' "UTF-16LE", "UTF-16"). #' #' @rdname column_string_functions @@ -855,7 +853,7 @@ setMethod("hex", }) #' @details -#' \code{hour}: Extracts the hours as an integer from a given date/timestamp/string. +#' \code{hour}: Extracts the hour as an integer from a given date/timestamp/string. #' #' @rdname column_datetime_functions #' @aliases hour hour,Column-method @@ -1177,7 +1175,7 @@ setMethod("min", }) #' @details -#' \code{minute}: Extracts the minutes as an integer from a given date/timestamp/string. +#' \code{minute}: Extracts the minute as an integer from a given date/timestamp/string. #' #' @rdname column_datetime_functions #' @aliases minute minute,Column-method @@ -1354,7 +1352,7 @@ setMethod("sd", }) #' @details -#' \code{second}: Extracts the seconds as an integer from a given date/timestamp/string. +#' \code{second}: Extracts the second as an integer from a given date/timestamp/string. #' #' @rdname column_datetime_functions #' @aliases second second,Column-method @@ -1464,20 +1462,18 @@ setMethod("soundex", column(jc) }) -#' Return the partition ID as a column -#' -#' Return the partition ID as a SparkDataFrame column. +#' @details +#' \code{spark_partition_id}: Returns the partition ID as a SparkDataFrame column. #' Note that this is nondeterministic because it depends on data partitioning and #' task scheduling. +#' This is equivalent to the \code{SPARK_PARTITION_ID} function in SQL. #' -#' This is equivalent to the SPARK_PARTITION_ID function in SQL. -#' -#' @rdname spark_partition_id -#' @name spark_partition_id -#' @aliases spark_partition_id,missing-method +#' @rdname column_nonaggregate_functions +#' @aliases spark_partition_id spark_partition_id,missing-method #' @export #' @examples -#' \dontrun{select(df, spark_partition_id())} +#' +#' \dontrun{head(select(df, spark_partition_id()))} #' @note spark_partition_id since 2.0.0 setMethod("spark_partition_id", signature("missing"), @@ -2028,7 +2024,7 @@ setMethod("pmod", signature(y = "Column"), column(jc) }) -#' @param rsd maximum estimation error allowed (default = 0.05) +#' @param rsd maximum estimation error allowed (default = 0.05). #' #' @rdname column_aggregate_functions #' @aliases approxCountDistinct,Column-method @@ -2220,8 +2216,8 @@ setMethod("from_json", signature(x = "Column", schema = "structType"), #' @examples #' #' \dontrun{ -#' tmp <- mutate(df, from_utc = from_utc_timestamp(df$time, 'PST'), -#' to_utc = to_utc_timestamp(df$time, 'PST')) +#' tmp <- mutate(df, from_utc = from_utc_timestamp(df$time, "PST"), +#' to_utc = to_utc_timestamp(df$time, "PST")) #' head(tmp)} #' @note from_utc_timestamp since 1.5.0 setMethod("from_utc_timestamp", signature(y = "Column", x = "character"), @@ -2255,7 +2251,7 @@ setMethod("instr", signature(y = "Column", x = "character"), #' @details #' \code{next_day}: Given a date column, returns the first date which is later than the value of #' the date column that is on the specified day of the week. For example, -#' \code{next_day('2015-07-27', "Sunday")} returns 2015-08-02 because that is the first Sunday +#' \code{next_day("2015-07-27", "Sunday")} returns 2015-08-02 because that is the first Sunday #' after 2015-07-27. Day of the week parameter is case insensitive, and accepts first three or #' two characters: "Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun". #' @@ -2295,7 +2291,7 @@ setMethod("to_utc_timestamp", signature(y = "Column", x = "character"), #' tmp <- mutate(df, t1 = add_months(df$time, 1), #' t2 = date_add(df$time, 2), #' t3 = date_sub(df$time, 3), -#' t4 = next_day(df$time, 'Sun')) +#' t4 = next_day(df$time, "Sun")) #' head(tmp)} #' @note add_months since 1.5.0 setMethod("add_months", signature(y = "Column", x = "numeric"), @@ -2404,8 +2400,8 @@ setMethod("shiftRight", signature(y = "Column", x = "numeric"), }) #' @details -#' \code{shiftRight}: (Unigned) shifts the given value numBits right. If the given value is a long value, -#' it will return a long value else it will return an integer value. +#' \code{shiftRightUnsigned}: (Unigned) shifts the given value numBits right. If the given value is +#' a long value, it will return a long value else it will return an integer value. #' #' @rdname column_math_functions #' @aliases shiftRightUnsigned shiftRightUnsigned,Column,numeric-method @@ -2513,14 +2509,13 @@ setMethod("from_unixtime", signature(x = "Column"), column(jc) }) -#' window -#' -#' Bucketize rows into one or more time windows given a timestamp specifying column. Window -#' starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window +#' @details +#' \code{window}: Bucketizes rows into one or more time windows given a timestamp specifying column. +#' Window starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window #' [12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in -#' the order of months are not supported. +#' the order of months are not supported. It returns an output column of struct called 'window' +#' by default with the nested columns 'start' and 'end' #' -#' @param x a time Column. Must be of TimestampType. #' @param windowDuration a string specifying the width of the window, e.g. '1 second', #' '1 day 12 hours', '2 minutes'. Valid interval strings are 'week', #' 'day', 'hour', 'minute', 'second', 'millisecond', 'microsecond'. Note that @@ -2536,27 +2531,22 @@ setMethod("from_unixtime", signature(x = "Column"), #' window intervals. For example, in order to have hourly tumbling windows #' that start 15 minutes past the hour, e.g. 12:15-13:15, 13:15-14:15... provide #' \code{startTime} as \code{"15 minutes"}. -#' @param ... further arguments to be passed to or from other methods. -#' @return An output column of struct called 'window' by default with the nested columns 'start' -#' and 'end'. -#' @family date time functions -#' @rdname window -#' @name window -#' @aliases window,Column-method +#' @rdname column_datetime_functions +#' @aliases window window,Column-method #' @export #' @examples -#'\dontrun{ -#' # One minute windows every 15 seconds 10 seconds after the minute, e.g. 09:00:10-09:01:10, -#' # 09:00:25-09:01:25, 09:00:40-09:01:40, ... -#' window(df$time, "1 minute", "15 seconds", "10 seconds") #' -#' # One minute tumbling windows 15 seconds after the minute, e.g. 09:00:15-09:01:15, -#' # 09:01:15-09:02:15... -#' window(df$time, "1 minute", startTime = "15 seconds") +#' \dontrun{ +#' # One minute windows every 15 seconds 10 seconds after the minute, e.g. 09:00:10-09:01:10, +#' # 09:00:25-09:01:25, 09:00:40-09:01:40, ... +#' window(df$time, "1 minute", "15 seconds", "10 seconds") #' -#' # Thirty-second windows every 10 seconds, e.g. 09:00:00-09:00:30, 09:00:10-09:00:40, ... -#' window(df$time, "30 seconds", "10 seconds") -#'} +#' # One minute tumbling windows 15 seconds after the minute, e.g. 09:00:15-09:01:15, +#' # 09:01:15-09:02:15... +#' window(df$time, "1 minute", startTime = "15 seconds") +#' +#' # Thirty-second windows every 10 seconds, e.g. 09:00:00-09:00:30, 09:00:10-09:00:40, ... +#' window(df$time, "30 seconds", "10 seconds")} #' @note window since 2.0.0 setMethod("window", signature(x = "Column"), function(x, windowDuration, slideDuration = NULL, startTime = NULL) { @@ -3046,7 +3036,7 @@ setMethod("row_number", #' \code{array_contains}: Returns null if the array is null, true if the array contains #' the value, and false otherwise. #' -#' @param value A value to be checked if contained in the column +#' @param value a value to be checked if contained in the column #' @rdname column_collection_functions #' @aliases array_contains array_contains,Column-method #' @export @@ -3091,7 +3081,7 @@ setMethod("size", #' to the natural ordering of the array elements. #' #' @rdname column_collection_functions -#' @param asc A logical flag indicating the sorting order. +#' @param asc a logical flag indicating the sorting order. #' TRUE, sorting is in ascending order. #' FALSE, sorting is in descending order. #' @aliases sort_array sort_array,Column-method @@ -3218,7 +3208,7 @@ setMethod("split_string", #' \code{repeat_string}: Repeats string n times. #' Equivalent to \code{repeat} SQL function. #' -#' @param n Number of repetitions +#' @param n number of repetitions. #' @rdname column_string_functions #' @aliases repeat_string repeat_string,Column-method #' @export @@ -3347,7 +3337,7 @@ setMethod("grouping_bit", #' \code{grouping_id}: Returns the level of grouping. #' Equals to \code{ #' grouping_bit(c1) * 2^(n - 1) + grouping_bit(c2) * 2^(n - 2) + ... + grouping_bit(cn) -#' } +#' }. #' #' @rdname column_aggregate_functions #' @aliases grouping_id grouping_id,Column-method diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index beac18e41273..92098741f72f 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1418,9 +1418,9 @@ setGeneric("split_string", function(x, pattern) { standardGeneric("split_string" #' @name NULL setGeneric("soundex", function(x) { standardGeneric("soundex") }) -#' @param x empty. Should be used with no argument. -#' @rdname spark_partition_id +#' @rdname column_nonaggregate_functions #' @export +#' @name NULL setGeneric("spark_partition_id", function(x = "missing") { standardGeneric("spark_partition_id") }) #' @rdname column_aggregate_functions @@ -1538,8 +1538,9 @@ setGeneric("var_samp", function(x) { standardGeneric("var_samp") }) #' @name NULL setGeneric("weekofyear", function(x) { standardGeneric("weekofyear") }) -#' @rdname window +#' @rdname column_datetime_functions #' @export +#' @name NULL setGeneric("window", function(x, ...) { standardGeneric("window") }) #' @rdname column_datetime_functions From f2c3b1dd69423cf52880e0ffa5f673ad6041b40e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 5 Jul 2017 14:17:26 +0800 Subject: [PATCH 037/125] [SPARK-21304][SQL] remove unnecessary isNull variable for collection related encoder expressions ## What changes were proposed in this pull request? For these collection-related encoder expressions, we don't need to create `isNull` variable if the loop element is not nullable. ## How was this patch tested? existing tests. Author: Wenchen Fan Closes #18529 from cloud-fan/minor. --- .../spark/sql/catalyst/ScalaReflection.scala | 2 +- .../expressions/objects/objects.scala | 77 ++++++++++++------- 2 files changed, 50 insertions(+), 29 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index f3c1e4150017..bea0de4d90c2 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 @@ -335,7 +335,7 @@ object ScalaReflection extends ScalaReflection { // TODO: add walked type path for map val TypeRef(_, _, Seq(keyType, valueType)) = t - CollectObjectsToMap( + CatalystToExternalMap( p => deserializerFor(keyType, Some(p), walkedTypePath), p => deserializerFor(valueType, Some(p), walkedTypePath), getPath, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index d6d06aecc077..ce07f4a25c18 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -465,7 +465,11 @@ object MapObjects { customCollectionCls: Option[Class[_]] = None): MapObjects = { val id = curId.getAndIncrement() val loopValue = s"MapObjects_loopValue$id" - val loopIsNull = s"MapObjects_loopIsNull$id" + val loopIsNull = if (elementNullable) { + s"MapObjects_loopIsNull$id" + } else { + "false" + } val loopVar = LambdaVariable(loopValue, loopIsNull, elementType, elementNullable) MapObjects( loopValue, loopIsNull, elementType, function(loopVar), inputData, customCollectionCls) @@ -517,7 +521,6 @@ case class MapObjects private( override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val elementJavaType = ctx.javaType(loopVarDataType) - ctx.addMutableState("boolean", loopIsNull, "") ctx.addMutableState(elementJavaType, loopValue, "") val genInputData = inputData.genCode(ctx) val genFunction = lambdaFunction.genCode(ctx) @@ -588,12 +591,14 @@ case class MapObjects private( case _ => genFunction.value } - val loopNullCheck = inputDataType match { - case _: ArrayType => s"$loopIsNull = ${genInputData.value}.isNullAt($loopIndex);" - // The element of primitive array will never be null. - case ObjectType(cls) if cls.isArray && cls.getComponentType.isPrimitive => - s"$loopIsNull = false" - case _ => s"$loopIsNull = $loopValue == null;" + val loopNullCheck = if (loopIsNull != "false") { + ctx.addMutableState("boolean", loopIsNull, "") + inputDataType match { + case _: ArrayType => s"$loopIsNull = ${genInputData.value}.isNullAt($loopIndex);" + case _ => s"$loopIsNull = $loopValue == null;" + } + } else { + "" } val (initCollection, addElement, getResult): (String, String => String, String) = @@ -667,11 +672,11 @@ case class MapObjects private( } } -object CollectObjectsToMap { +object CatalystToExternalMap { private val curId = new java.util.concurrent.atomic.AtomicInteger() /** - * Construct an instance of CollectObjectsToMap case class. + * Construct an instance of CatalystToExternalMap case class. * * @param keyFunction The function applied on the key collection elements. * @param valueFunction The function applied on the value collection elements. @@ -682,15 +687,19 @@ object CollectObjectsToMap { keyFunction: Expression => Expression, valueFunction: Expression => Expression, inputData: Expression, - collClass: Class[_]): CollectObjectsToMap = { + collClass: Class[_]): CatalystToExternalMap = { val id = curId.getAndIncrement() - val keyLoopValue = s"CollectObjectsToMap_keyLoopValue$id" + val keyLoopValue = s"CatalystToExternalMap_keyLoopValue$id" val mapType = inputData.dataType.asInstanceOf[MapType] val keyLoopVar = LambdaVariable(keyLoopValue, "", mapType.keyType, nullable = false) - val valueLoopValue = s"CollectObjectsToMap_valueLoopValue$id" - val valueLoopIsNull = s"CollectObjectsToMap_valueLoopIsNull$id" + val valueLoopValue = s"CatalystToExternalMap_valueLoopValue$id" + val valueLoopIsNull = if (mapType.valueContainsNull) { + s"CatalystToExternalMap_valueLoopIsNull$id" + } else { + "false" + } val valueLoopVar = LambdaVariable(valueLoopValue, valueLoopIsNull, mapType.valueType) - CollectObjectsToMap( + CatalystToExternalMap( keyLoopValue, keyFunction(keyLoopVar), valueLoopValue, valueLoopIsNull, valueFunction(valueLoopVar), inputData, collClass) @@ -716,7 +725,7 @@ object CollectObjectsToMap { * @param inputData An expression that when evaluated returns a map object. * @param collClass The type of the resulting collection. */ -case class CollectObjectsToMap private( +case class CatalystToExternalMap private( keyLoopValue: String, keyLambdaFunction: Expression, valueLoopValue: String, @@ -748,7 +757,6 @@ case class CollectObjectsToMap private( ctx.addMutableState(keyElementJavaType, keyLoopValue, "") val genKeyFunction = keyLambdaFunction.genCode(ctx) val valueElementJavaType = ctx.javaType(mapType.valueType) - ctx.addMutableState("boolean", valueLoopIsNull, "") ctx.addMutableState(valueElementJavaType, valueLoopValue, "") val genValueFunction = valueLambdaFunction.genCode(ctx) val genInputData = inputData.genCode(ctx) @@ -781,7 +789,12 @@ case class CollectObjectsToMap private( val genKeyFunctionValue = genFunctionValue(keyLambdaFunction, genKeyFunction) val genValueFunctionValue = genFunctionValue(valueLambdaFunction, genValueFunction) - val valueLoopNullCheck = s"$valueLoopIsNull = $valueArray.isNullAt($loopIndex);" + val valueLoopNullCheck = if (valueLoopIsNull != "false") { + ctx.addMutableState("boolean", valueLoopIsNull, "") + s"$valueLoopIsNull = $valueArray.isNullAt($loopIndex);" + } else { + "" + } val builderClass = classOf[Builder[_, _]].getName val constructBuilder = s""" @@ -847,9 +860,17 @@ object ExternalMapToCatalyst { valueNullable: Boolean): ExternalMapToCatalyst = { val id = curId.getAndIncrement() val keyName = "ExternalMapToCatalyst_key" + id - val keyIsNull = "ExternalMapToCatalyst_key_isNull" + id + val keyIsNull = if (keyNullable) { + "ExternalMapToCatalyst_key_isNull" + id + } else { + "false" + } val valueName = "ExternalMapToCatalyst_value" + id - val valueIsNull = "ExternalMapToCatalyst_value_isNull" + id + val valueIsNull = if (valueNullable) { + "ExternalMapToCatalyst_value_isNull" + id + } else { + "false" + } ExternalMapToCatalyst( keyName, @@ -919,9 +940,7 @@ case class ExternalMapToCatalyst private( val keyElementJavaType = ctx.javaType(keyType) val valueElementJavaType = ctx.javaType(valueType) - ctx.addMutableState("boolean", keyIsNull, "") ctx.addMutableState(keyElementJavaType, key, "") - ctx.addMutableState("boolean", valueIsNull, "") ctx.addMutableState(valueElementJavaType, value, "") val (defineEntries, defineKeyValue) = child.dataType match { @@ -957,16 +976,18 @@ case class ExternalMapToCatalyst private( defineEntries -> defineKeyValue } - val keyNullCheck = if (ctx.isPrimitiveType(keyType)) { - s"$keyIsNull = false;" - } else { + val keyNullCheck = if (keyIsNull != "false") { + ctx.addMutableState("boolean", keyIsNull, "") s"$keyIsNull = $key == null;" + } else { + "" } - val valueNullCheck = if (ctx.isPrimitiveType(valueType)) { - s"$valueIsNull = false;" - } else { + val valueNullCheck = if (valueIsNull != "false") { + ctx.addMutableState("boolean", valueIsNull, "") s"$valueIsNull = $value == null;" + } else { + "" } val arrayCls = classOf[GenericArrayData].getName From a38643256691947ff7f7c474b85c052a7d5d8553 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Wed, 5 Jul 2017 14:25:26 +0800 Subject: [PATCH 038/125] [SPARK-18623][SQL] Add `returnNullable` to `StaticInvoke` and modify it to handle properly. ## What changes were proposed in this pull request? Add `returnNullable` to `StaticInvoke` the same as #15780 is trying to add to `Invoke` and modify to handle properly. ## How was this patch tested? Existing tests. Author: Takuya UESHIN Author: Takuya UESHIN Closes #16056 from ueshin/issues/SPARK-18623. --- .../sql/catalyst/JavaTypeInference.scala | 21 +++++---- .../spark/sql/catalyst/ScalaReflection.scala | 44 ++++++++++++------- .../sql/catalyst/encoders/RowEncoder.scala | 27 ++++++++---- .../expressions/objects/objects.scala | 42 ++++++++++++++---- 4 files changed, 91 insertions(+), 43 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index 90ec699877de..21363d3ba82c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -216,7 +216,7 @@ object JavaTypeInference { ObjectType(c), "valueOf", getPath :: Nil, - propagateNull = true) + returnNullable = false) case c if c == classOf[java.sql.Date] => StaticInvoke( @@ -224,7 +224,7 @@ object JavaTypeInference { ObjectType(c), "toJavaDate", getPath :: Nil, - propagateNull = true) + returnNullable = false) case c if c == classOf[java.sql.Timestamp] => StaticInvoke( @@ -232,7 +232,7 @@ object JavaTypeInference { ObjectType(c), "toJavaTimestamp", getPath :: Nil, - propagateNull = true) + returnNullable = false) case c if c == classOf[java.lang.String] => Invoke(getPath, "toString", ObjectType(classOf[String])) @@ -300,7 +300,8 @@ object JavaTypeInference { ArrayBasedMapData.getClass, ObjectType(classOf[JMap[_, _]]), "toJavaMap", - keyData :: valueData :: Nil) + keyData :: valueData :: Nil, + returnNullable = false) case other => val properties = getJavaBeanReadableAndWritableProperties(other) @@ -367,28 +368,32 @@ object JavaTypeInference { classOf[UTF8String], StringType, "fromString", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case c if c == classOf[java.sql.Timestamp] => StaticInvoke( DateTimeUtils.getClass, TimestampType, "fromJavaTimestamp", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case c if c == classOf[java.sql.Date] => StaticInvoke( DateTimeUtils.getClass, DateType, "fromJavaDate", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case c if c == classOf[java.math.BigDecimal] => StaticInvoke( Decimal.getClass, DecimalType.SYSTEM_DEFAULT, "apply", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case c if c == classOf[java.lang.Boolean] => Invoke(inputObject, "booleanValue", BooleanType) 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 bea0de4d90c2..814f2c10b909 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 @@ -206,51 +206,53 @@ object ScalaReflection extends ScalaReflection { case t if t <:< localTypeOf[java.lang.Integer] => val boxedType = classOf[java.lang.Integer] val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, propagateNull = true) + StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.lang.Long] => val boxedType = classOf[java.lang.Long] val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, propagateNull = true) + StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.lang.Double] => val boxedType = classOf[java.lang.Double] val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, propagateNull = true) + StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.lang.Float] => val boxedType = classOf[java.lang.Float] val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, propagateNull = true) + StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.lang.Short] => val boxedType = classOf[java.lang.Short] val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, propagateNull = true) + StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.lang.Byte] => val boxedType = classOf[java.lang.Byte] val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, propagateNull = true) + StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.lang.Boolean] => val boxedType = classOf[java.lang.Boolean] val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, propagateNull = true) + StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.sql.Date] => StaticInvoke( DateTimeUtils.getClass, ObjectType(classOf[java.sql.Date]), "toJavaDate", - getPath :: Nil) + getPath :: Nil, + returnNullable = false) case t if t <:< localTypeOf[java.sql.Timestamp] => StaticInvoke( DateTimeUtils.getClass, ObjectType(classOf[java.sql.Timestamp]), "toJavaTimestamp", - getPath :: Nil) + getPath :: Nil, + returnNullable = false) case t if t <:< localTypeOf[java.lang.String] => Invoke(getPath, "toString", ObjectType(classOf[String]), returnNullable = false) @@ -446,7 +448,8 @@ object ScalaReflection extends ScalaReflection { classOf[UnsafeArrayData], ArrayType(dt, false), "fromPrimitiveArray", - input :: Nil) + input :: Nil, + returnNullable = false) } else { NewInstance( classOf[GenericArrayData], @@ -504,49 +507,56 @@ object ScalaReflection extends ScalaReflection { classOf[UTF8String], StringType, "fromString", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case t if t <:< localTypeOf[java.sql.Timestamp] => StaticInvoke( DateTimeUtils.getClass, TimestampType, "fromJavaTimestamp", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case t if t <:< localTypeOf[java.sql.Date] => StaticInvoke( DateTimeUtils.getClass, DateType, "fromJavaDate", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case t if t <:< localTypeOf[BigDecimal] => StaticInvoke( Decimal.getClass, DecimalType.SYSTEM_DEFAULT, "apply", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case t if t <:< localTypeOf[java.math.BigDecimal] => StaticInvoke( Decimal.getClass, DecimalType.SYSTEM_DEFAULT, "apply", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case t if t <:< localTypeOf[java.math.BigInteger] => StaticInvoke( Decimal.getClass, DecimalType.BigIntDecimal, "apply", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case t if t <:< localTypeOf[scala.math.BigInt] => StaticInvoke( Decimal.getClass, DecimalType.BigIntDecimal, "apply", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case t if t <:< localTypeOf[java.lang.Integer] => Invoke(inputObject, "intValue", IntegerType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 0f8282d3b2f1..cc32fac67e92 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -96,28 +96,32 @@ object RowEncoder { DateTimeUtils.getClass, TimestampType, "fromJavaTimestamp", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case DateType => StaticInvoke( DateTimeUtils.getClass, DateType, "fromJavaDate", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case d: DecimalType => StaticInvoke( Decimal.getClass, d, "fromDecimal", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case StringType => StaticInvoke( classOf[UTF8String], StringType, "fromString", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case t @ ArrayType(et, cn) => et match { @@ -126,7 +130,8 @@ object RowEncoder { classOf[ArrayData], t, "toArrayData", - inputObject :: Nil) + inputObject :: Nil, + returnNullable = false) case _ => MapObjects( element => serializerFor(ValidateExternalType(element, et), et), inputObject, @@ -254,14 +259,16 @@ object RowEncoder { DateTimeUtils.getClass, ObjectType(classOf[java.sql.Timestamp]), "toJavaTimestamp", - input :: Nil) + input :: Nil, + returnNullable = false) case DateType => StaticInvoke( DateTimeUtils.getClass, ObjectType(classOf[java.sql.Date]), "toJavaDate", - input :: Nil) + input :: Nil, + returnNullable = false) case _: DecimalType => Invoke(input, "toJavaBigDecimal", ObjectType(classOf[java.math.BigDecimal]), @@ -280,7 +287,8 @@ object RowEncoder { scala.collection.mutable.WrappedArray.getClass, ObjectType(classOf[Seq[_]]), "make", - arrayData :: Nil) + arrayData :: Nil, + returnNullable = false) case MapType(kt, vt, valueNullable) => val keyArrayType = ArrayType(kt, false) @@ -293,7 +301,8 @@ object RowEncoder { ArrayBasedMapData.getClass, ObjectType(classOf[Map[_, _]]), "toScalaMap", - keyData :: valueData :: Nil) + keyData :: valueData :: Nil, + returnNullable = false) case schema @ StructType(fields) => val convertedFields = fields.zipWithIndex.map { case (f, i) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index ce07f4a25c18..24c06d8b14b5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -118,17 +118,20 @@ trait InvokeLike extends Expression with NonSQLExpression { * @param arguments An optional list of expressions to pass as arguments to the function. * @param propagateNull When true, and any of the arguments is null, null will be returned instead * of calling the function. + * @param returnNullable When false, indicating the invoked method will always return + * non-null value. */ case class StaticInvoke( staticObject: Class[_], dataType: DataType, functionName: String, arguments: Seq[Expression] = Nil, - propagateNull: Boolean = true) extends InvokeLike { + propagateNull: Boolean = true, + returnNullable: Boolean = true) extends InvokeLike { val objectName = staticObject.getName.stripSuffix("$") - override def nullable: Boolean = true + override def nullable: Boolean = needNullCheck || returnNullable override def children: Seq[Expression] = arguments override def eval(input: InternalRow): Any = @@ -141,19 +144,40 @@ case class StaticInvoke( val callFunc = s"$objectName.$functionName($argString)" - // If the function can return null, we do an extra check to make sure our null bit is still set - // correctly. - val postNullCheck = if (ctx.defaultValue(dataType) == "null") { - s"${ev.isNull} = ${ev.value} == null;" + val prepareIsNull = if (nullable) { + s"boolean ${ev.isNull} = $resultIsNull;" } else { + ev.isNull = "false" "" } + val evaluate = if (returnNullable) { + if (ctx.defaultValue(dataType) == "null") { + s""" + ${ev.value} = $callFunc; + ${ev.isNull} = ${ev.value} == null; + """ + } else { + val boxedResult = ctx.freshName("boxedResult") + s""" + ${ctx.boxedType(dataType)} $boxedResult = $callFunc; + ${ev.isNull} = $boxedResult == null; + if (!${ev.isNull}) { + ${ev.value} = $boxedResult; + } + """ + } + } else { + s"${ev.value} = $callFunc;" + } + val code = s""" $argCode - boolean ${ev.isNull} = $resultIsNull; - final $javaType ${ev.value} = $resultIsNull ? ${ctx.defaultValue(dataType)} : $callFunc; - $postNullCheck + $prepareIsNull + $javaType ${ev.value} = ${ctx.defaultValue(dataType)}; + if (!$resultIsNull) { + $evaluate + } """ ev.copy(code = code) } From 4852b7d447e872079c2c81428354adc825a87b27 Mon Sep 17 00:00:00 2001 From: actuaryzhang Date: Wed, 5 Jul 2017 18:41:00 +0800 Subject: [PATCH 039/125] [SPARK-21310][ML][PYSPARK] Expose offset in PySpark ## What changes were proposed in this pull request? Add offset to PySpark in GLM as in #16699. ## How was this patch tested? Python test Author: actuaryzhang Closes #18534 from actuaryzhang/pythonOffset. --- python/pyspark/ml/regression.py | 25 +++++++++++++++++++++---- python/pyspark/ml/tests.py | 14 ++++++++++++++ 2 files changed, 35 insertions(+), 4 deletions(-) diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 84d843369e10..f0ff7a5f59ab 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -1376,17 +1376,20 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha typeConverter=TypeConverters.toFloat) solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + "options: irls.", typeConverter=TypeConverters.toString) + offsetCol = Param(Params._dummy(), "offsetCol", "The offset column name. If this is not set " + + "or empty, we treat all instance offsets as 0.0", + typeConverter=TypeConverters.toString) @keyword_only def __init__(self, labelCol="label", featuresCol="features", predictionCol="prediction", family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=None, - variancePower=0.0, linkPower=None): + variancePower=0.0, linkPower=None, offsetCol=None): """ __init__(self, labelCol="label", featuresCol="features", predictionCol="prediction", \ family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, \ regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=None, \ - variancePower=0.0, linkPower=None) + variancePower=0.0, linkPower=None, offsetCol=None) """ super(GeneralizedLinearRegression, self).__init__() self._java_obj = self._new_java_obj( @@ -1402,12 +1405,12 @@ def __init__(self, labelCol="label", featuresCol="features", predictionCol="pred def setParams(self, labelCol="label", featuresCol="features", predictionCol="prediction", family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=None, - variancePower=0.0, linkPower=None): + variancePower=0.0, linkPower=None, offsetCol=None): """ setParams(self, labelCol="label", featuresCol="features", predictionCol="prediction", \ family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, \ regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=None, \ - variancePower=0.0, linkPower=None) + variancePower=0.0, linkPower=None, offsetCol=None) Sets params for generalized linear regression. """ kwargs = self._input_kwargs @@ -1486,6 +1489,20 @@ def getLinkPower(self): """ return self.getOrDefault(self.linkPower) + @since("2.3.0") + def setOffsetCol(self, value): + """ + Sets the value of :py:attr:`offsetCol`. + """ + return self._set(offsetCol=value) + + @since("2.3.0") + def getOffsetCol(self): + """ + Gets the value of offsetCol or its default value. + """ + return self.getOrDefault(self.offsetCol) + class GeneralizedLinearRegressionModel(JavaModel, JavaPredictionModel, JavaMLWritable, JavaMLReadable): diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index ffb8b0a890ff..787004765160 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -1291,6 +1291,20 @@ def test_tweedie_distribution(self): self.assertTrue(np.allclose(model2.coefficients.toArray(), [-0.6667, 0.5], atol=1E-4)) self.assertTrue(np.isclose(model2.intercept, 0.6667, atol=1E-4)) + def test_offset(self): + + df = self.spark.createDataFrame( + [(0.2, 1.0, 2.0, Vectors.dense(0.0, 5.0)), + (0.5, 2.1, 0.5, Vectors.dense(1.0, 2.0)), + (0.9, 0.4, 1.0, Vectors.dense(2.0, 1.0)), + (0.7, 0.7, 0.0, Vectors.dense(3.0, 3.0))], ["label", "weight", "offset", "features"]) + + glr = GeneralizedLinearRegression(family="poisson", weightCol="weight", offsetCol="offset") + model = glr.fit(df) + self.assertTrue(np.allclose(model.coefficients.toArray(), [0.664647, -0.3192581], + atol=1E-4)) + self.assertTrue(np.isclose(model.intercept, -1.561613, atol=1E-4)) + class FPGrowthTests(SparkSessionTestCase): def setUp(self): From 873f3ad2b89c955f42fced49dc129e8efa77d044 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Wed, 5 Jul 2017 20:32:47 +0800 Subject: [PATCH 040/125] [SPARK-16167][SQL] RowEncoder should preserve array/map type nullability. ## What changes were proposed in this pull request? Currently `RowEncoder` doesn't preserve nullability of `ArrayType` or `MapType`. It returns always `containsNull = true` for `ArrayType`, `valueContainsNull = true` for `MapType` and also the nullability of itself is always `true`. This pr fixes the nullability of them. ## How was this patch tested? Add tests to check if `RowEncoder` preserves array/map nullability. Author: Takuya UESHIN Author: Takuya UESHIN Closes #13873 from ueshin/issues/SPARK-16167. --- .../sql/catalyst/encoders/RowEncoder.scala | 25 +++++++++++--- .../catalyst/encoders/RowEncoderSuite.scala | 33 +++++++++++++++++++ 2 files changed, 54 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index cc32fac67e92..43c35bbdf383 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -123,7 +123,7 @@ object RowEncoder { inputObject :: Nil, returnNullable = false) - case t @ ArrayType(et, cn) => + case t @ ArrayType(et, containsNull) => et match { case BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType => StaticInvoke( @@ -132,8 +132,16 @@ object RowEncoder { "toArrayData", inputObject :: Nil, returnNullable = false) + case _ => MapObjects( - element => serializerFor(ValidateExternalType(element, et), et), + element => { + val value = serializerFor(ValidateExternalType(element, et), et) + if (!containsNull) { + AssertNotNull(value, Seq.empty) + } else { + value + } + }, inputObject, ObjectType(classOf[Object])) } @@ -155,10 +163,19 @@ object RowEncoder { ObjectType(classOf[scala.collection.Seq[_]]), returnNullable = false) val convertedValues = serializerFor(values, ArrayType(vt, valueNullable)) - NewInstance( + val nonNullOutput = NewInstance( classOf[ArrayBasedMapData], convertedKeys :: convertedValues :: Nil, - dataType = t) + dataType = t, + propagateNull = false) + + if (inputObject.nullable) { + If(IsNull(inputObject), + Literal.create(null, inputType), + nonNullOutput) + } else { + nonNullOutput + } case StructType(fields) => val nonNullOutput = CreateNamedStruct(fields.zipWithIndex.flatMap { case (field, index) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index 1a5569a77dc7..6ed175f86ca7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -273,6 +273,39 @@ class RowEncoderSuite extends SparkFunSuite { assert(e4.getMessage.contains("java.lang.String is not a valid external type")) } + for { + elementType <- Seq(IntegerType, StringType) + containsNull <- Seq(true, false) + nullable <- Seq(true, false) + } { + test("RowEncoder should preserve array nullability: " + + s"ArrayType($elementType, containsNull = $containsNull), nullable = $nullable") { + val schema = new StructType().add("array", ArrayType(elementType, containsNull), nullable) + val encoder = RowEncoder(schema).resolveAndBind() + assert(encoder.serializer.length == 1) + assert(encoder.serializer.head.dataType == ArrayType(elementType, containsNull)) + assert(encoder.serializer.head.nullable == nullable) + } + } + + for { + keyType <- Seq(IntegerType, StringType) + valueType <- Seq(IntegerType, StringType) + valueContainsNull <- Seq(true, false) + nullable <- Seq(true, false) + } { + test("RowEncoder should preserve map nullability: " + + s"MapType($keyType, $valueType, valueContainsNull = $valueContainsNull), " + + s"nullable = $nullable") { + val schema = new StructType().add( + "map", MapType(keyType, valueType, valueContainsNull), nullable) + val encoder = RowEncoder(schema).resolveAndBind() + assert(encoder.serializer.length == 1) + assert(encoder.serializer.head.dataType == MapType(keyType, valueType, valueContainsNull)) + assert(encoder.serializer.head.nullable == nullable) + } + } + private def encodeDecodeTest(schema: StructType): Unit = { test(s"encode/decode: ${schema.simpleString}") { val encoder = RowEncoder(schema).resolveAndBind() From 5787ace463b2abde50d2ca24e8dd111e3a7c158e Mon Sep 17 00:00:00 2001 From: ouyangxiaochen Date: Wed, 5 Jul 2017 20:46:42 +0800 Subject: [PATCH 041/125] [SPARK-20383][SQL] Supporting Create [temporary] Function with the keyword 'OR REPLACE' and 'IF NOT EXISTS' ## What changes were proposed in this pull request? support to create [temporary] function with the keyword 'OR REPLACE' and 'IF NOT EXISTS' ## How was this patch tested? manual test and added test cases Please review http://spark.apache.org/contributing.html before opening a pull request. Author: ouyangxiaochen Closes #17681 from ouyangxiaochen/spark-419. --- .../spark/sql/catalyst/parser/SqlBase.g4 | 3 +- .../catalyst/catalog/ExternalCatalog.scala | 9 ++++ .../catalyst/catalog/InMemoryCatalog.scala | 6 +++ .../sql/catalyst/catalog/SessionCatalog.scala | 23 ++++++++ .../spark/sql/catalyst/catalog/events.scala | 10 ++++ .../catalog/ExternalCatalogEventSuite.scala | 9 ++++ .../catalog/ExternalCatalogSuite.scala | 9 ++++ .../spark/sql/execution/SparkSqlParser.scala | 8 +-- .../sql/execution/command/functions.scala | 46 +++++++++++----- .../execution/command/DDLCommandSuite.scala | 52 ++++++++++++++++++- .../sql/execution/command/DDLSuite.scala | 51 ++++++++++++++++++ .../spark/sql/hive/HiveExternalCatalog.scala | 9 ++++ 12 files changed, 216 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 29f554451ed4..ef9f88a9026c 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -126,7 +126,8 @@ statement tableIdentifier ('(' colTypeList ')')? tableProvider (OPTIONS tablePropertyList)? #createTempViewUsing | ALTER VIEW tableIdentifier AS? query #alterViewQuery - | CREATE TEMPORARY? FUNCTION qualifiedName AS className=STRING + | CREATE (OR REPLACE)? TEMPORARY? FUNCTION (IF NOT EXISTS)? + qualifiedName AS className=STRING (USING resource (',' resource)*)? #createFunction | DROP TEMPORARY? FUNCTION (IF EXISTS)? qualifiedName #dropFunction | EXPLAIN (LOGICAL | FORMATTED | EXTENDED | CODEGEN | COST)? diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala index 0254b6bb6d13..6000d483db20 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala @@ -332,6 +332,15 @@ abstract class ExternalCatalog protected def doDropFunction(db: String, funcName: String): Unit + final def alterFunction(db: String, funcDefinition: CatalogFunction): Unit = { + val name = funcDefinition.identifier.funcName + postToAll(AlterFunctionPreEvent(db, name)) + doAlterFunction(db, funcDefinition) + postToAll(AlterFunctionEvent(db, name)) + } + + protected def doAlterFunction(db: String, funcDefinition: CatalogFunction): Unit + final def renameFunction(db: String, oldName: String, newName: String): Unit = { postToAll(RenameFunctionPreEvent(db, oldName, newName)) doRenameFunction(db, oldName, newName) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 747190faa3c8..d253c72a6273 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -590,6 +590,12 @@ class InMemoryCatalog( catalog(db).functions.remove(funcName) } + override protected def doAlterFunction(db: String, func: CatalogFunction): Unit = synchronized { + requireDbExists(db) + requireFunctionExists(db, func.identifier.funcName) + catalog(db).functions.put(func.identifier.funcName, func) + } + override protected def doRenameFunction( db: String, oldName: String, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index a86604e4353a..c40d5f6031a2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -1055,6 +1055,29 @@ class SessionCatalog( } } + /** + * overwirte a metastore function in the database specified in `funcDefinition`.. + * If no database is specified, assume the function is in the current database. + */ + def alterFunction(funcDefinition: CatalogFunction): Unit = { + val db = formatDatabaseName(funcDefinition.identifier.database.getOrElse(getCurrentDatabase)) + requireDbExists(db) + val identifier = FunctionIdentifier(funcDefinition.identifier.funcName, Some(db)) + val newFuncDefinition = funcDefinition.copy(identifier = identifier) + if (functionExists(identifier)) { + if (functionRegistry.functionExists(identifier)) { + // If we have loaded this function into the FunctionRegistry, + // also drop it from there. + // For a permanent function, because we loaded it to the FunctionRegistry + // when it's first used, we also need to drop it from the FunctionRegistry. + functionRegistry.dropFunction(identifier) + } + externalCatalog.alterFunction(db, newFuncDefinition) + } else { + throw new NoSuchFunctionException(db = db, func = identifier.toString) + } + } + /** * Retrieve the metadata of a metastore function. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/events.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/events.scala index 459973a13bb1..742a51e64038 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/events.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/events.scala @@ -139,6 +139,16 @@ case class DropFunctionPreEvent(database: String, name: String) extends Function */ case class DropFunctionEvent(database: String, name: String) extends FunctionEvent +/** + * Event fired before a function is altered. + */ +case class AlterFunctionPreEvent(database: String, name: String) extends FunctionEvent + +/** + * Event fired after a function has been altered. + */ +case class AlterFunctionEvent(database: String, name: String) extends FunctionEvent + /** * Event fired before a function is renamed. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogEventSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogEventSuite.scala index 2539ea615ff9..087c26f23f38 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogEventSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogEventSuite.scala @@ -176,6 +176,15 @@ class ExternalCatalogEventSuite extends SparkFunSuite { } checkEvents(RenameFunctionPreEvent("db5", "fn7", "fn4") :: Nil) + // ALTER + val alteredFunctionDefinition = CatalogFunction( + identifier = FunctionIdentifier("fn4", Some("db5")), + className = "org.apache.spark.AlterFunction", + resources = Seq.empty) + catalog.alterFunction("db5", alteredFunctionDefinition) + checkEvents( + AlterFunctionPreEvent("db5", "fn4") :: AlterFunctionEvent("db5", "fn4") :: Nil) + // DROP intercept[AnalysisException] { catalog.dropFunction("db5", "fn7") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index c22d55fc96a6..66e895a4690c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -752,6 +752,14 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac } } + test("alter function") { + val catalog = newBasicCatalog() + assert(catalog.getFunction("db2", "func1").className == funcClass) + val myNewFunc = catalog.getFunction("db2", "func1").copy(className = newFuncClass) + catalog.alterFunction("db2", myNewFunc) + assert(catalog.getFunction("db2", "func1").className == newFuncClass) + } + test("list functions") { val catalog = newBasicCatalog() catalog.createFunction("db2", newFunc("func2")) @@ -916,6 +924,7 @@ abstract class CatalogTestUtils { lazy val partWithEmptyValue = CatalogTablePartition(Map("a" -> "3", "b" -> ""), storageFormat) lazy val funcClass = "org.apache.spark.myFunc" + lazy val newFuncClass = "org.apache.spark.myNewFunc" /** * Creates a basic catalog, with the following structure: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 2b79eb5eac0f..2f8e416e7df1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -687,8 +687,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * * For example: * {{{ - * CREATE [TEMPORARY] FUNCTION [db_name.]function_name AS class_name - * [USING JAR|FILE|ARCHIVE 'file_uri' [, JAR|FILE|ARCHIVE 'file_uri']]; + * CREATE [OR REPLACE] [TEMPORARY] FUNCTION [IF NOT EXISTS] [db_name.]function_name + * AS class_name [USING JAR|FILE|ARCHIVE 'file_uri' [, JAR|FILE|ARCHIVE 'file_uri']]; * }}} */ override def visitCreateFunction(ctx: CreateFunctionContext): LogicalPlan = withOrigin(ctx) { @@ -709,7 +709,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { functionIdentifier.funcName, string(ctx.className), resources, - ctx.TEMPORARY != null) + ctx.TEMPORARY != null, + ctx.EXISTS != null, + ctx.REPLACE != null) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala index a91ad413f4d1..4f92ffee687a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala @@ -31,13 +31,13 @@ import org.apache.spark.sql.types.{StringType, StructField, StructType} * The DDL command that creates a function. * To create a temporary function, the syntax of using this command in SQL is: * {{{ - * CREATE TEMPORARY FUNCTION functionName + * CREATE [OR REPLACE] TEMPORARY FUNCTION functionName * AS className [USING JAR\FILE 'uri' [, JAR|FILE 'uri']] * }}} * * To create a permanent function, the syntax in SQL is: * {{{ - * CREATE FUNCTION [databaseName.]functionName + * CREATE [OR REPLACE] FUNCTION [IF NOT EXISTS] [databaseName.]functionName * AS className [USING JAR\FILE 'uri' [, JAR|FILE 'uri']] * }}} */ @@ -46,26 +46,46 @@ case class CreateFunctionCommand( functionName: String, className: String, resources: Seq[FunctionResource], - isTemp: Boolean) + isTemp: Boolean, + ifNotExists: Boolean, + replace: Boolean) extends RunnableCommand { + if (ifNotExists && replace) { + throw new AnalysisException("CREATE FUNCTION with both IF NOT EXISTS and REPLACE" + + " is not allowed.") + } + + // Disallow to define a temporary function with `IF NOT EXISTS` + if (ifNotExists && isTemp) { + throw new AnalysisException( + "It is not allowed to define a TEMPORARY function with IF NOT EXISTS.") + } + + // Temporary function names should not contain database prefix like "database.function" + if (databaseName.isDefined && isTemp) { + throw new AnalysisException(s"Specifying a database in CREATE TEMPORARY FUNCTION " + + s"is not allowed: '${databaseName.get}'") + } + override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val func = CatalogFunction(FunctionIdentifier(functionName, databaseName), className, resources) if (isTemp) { - if (databaseName.isDefined) { - throw new AnalysisException(s"Specifying a database in CREATE TEMPORARY FUNCTION " + - s"is not allowed: '${databaseName.get}'") - } // We first load resources and then put the builder in the function registry. catalog.loadFunctionResources(resources) - catalog.registerFunction(func, overrideIfExists = false) + catalog.registerFunction(func, overrideIfExists = replace) } else { - // For a permanent, we will store the metadata into underlying external catalog. - // This function will be loaded into the FunctionRegistry when a query uses it. - // We do not load it into FunctionRegistry right now. - // TODO: should we also parse "IF NOT EXISTS"? - catalog.createFunction(func, ignoreIfExists = false) + // Handles `CREATE OR REPLACE FUNCTION AS ... USING ...` + if (replace && catalog.functionExists(func.identifier)) { + // alter the function in the metastore + catalog.alterFunction(CatalogFunction(func.identifier, className, resources)) + } else { + // For a permanent, we will store the metadata into underlying external catalog. + // This function will be loaded into the FunctionRegistry when a query uses it. + // We do not load it into FunctionRegistry right now. + catalog.createFunction(CatalogFunction(func.identifier, className, resources), ifNotExists) + } } Seq.empty[Row] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 8a6bc62fec96..5643c58d9f84 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -181,8 +181,29 @@ class DDLCommandSuite extends PlanTest { |'com.matthewrathbone.example.SimpleUDFExample' USING ARCHIVE '/path/to/archive', |FILE '/path/to/file' """.stripMargin + val sql3 = + """ + |CREATE OR REPLACE TEMPORARY FUNCTION helloworld3 as + |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', + |JAR '/path/to/jar2' + """.stripMargin + val sql4 = + """ + |CREATE OR REPLACE FUNCTION hello.world1 as + |'com.matthewrathbone.example.SimpleUDFExample' USING ARCHIVE '/path/to/archive', + |FILE '/path/to/file' + """.stripMargin + val sql5 = + """ + |CREATE FUNCTION IF NOT EXISTS hello.world2 as + |'com.matthewrathbone.example.SimpleUDFExample' USING ARCHIVE '/path/to/archive', + |FILE '/path/to/file' + """.stripMargin val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + val parsed4 = parser.parsePlan(sql4) + val parsed5 = parser.parsePlan(sql5) val expected1 = CreateFunctionCommand( None, "helloworld", @@ -190,7 +211,7 @@ class DDLCommandSuite extends PlanTest { Seq( FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar1"), FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar2")), - isTemp = true) + isTemp = true, ifNotExists = false, replace = false) val expected2 = CreateFunctionCommand( Some("hello"), "world", @@ -198,9 +219,36 @@ class DDLCommandSuite extends PlanTest { Seq( FunctionResource(FunctionResourceType.fromString("archive"), "/path/to/archive"), FunctionResource(FunctionResourceType.fromString("file"), "/path/to/file")), - isTemp = false) + isTemp = false, ifNotExists = false, replace = false) + val expected3 = CreateFunctionCommand( + None, + "helloworld3", + "com.matthewrathbone.example.SimpleUDFExample", + Seq( + FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar1"), + FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar2")), + isTemp = true, ifNotExists = false, replace = true) + val expected4 = CreateFunctionCommand( + Some("hello"), + "world1", + "com.matthewrathbone.example.SimpleUDFExample", + Seq( + FunctionResource(FunctionResourceType.fromString("archive"), "/path/to/archive"), + FunctionResource(FunctionResourceType.fromString("file"), "/path/to/file")), + isTemp = false, ifNotExists = false, replace = true) + val expected5 = CreateFunctionCommand( + Some("hello"), + "world2", + "com.matthewrathbone.example.SimpleUDFExample", + Seq( + FunctionResource(FunctionResourceType.fromString("archive"), "/path/to/archive"), + FunctionResource(FunctionResourceType.fromString("file"), "/path/to/file")), + isTemp = false, ifNotExists = true, replace = false) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) + comparePlans(parsed5, expected5) } test("drop function") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index e4dd077715d0..5c40d8bb4b1e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -2270,6 +2270,57 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } + test("create temporary function with if not exists") { + withUserDefinedFunction("func1" -> true) { + val sql1 = + """ + |CREATE TEMPORARY FUNCTION IF NOT EXISTS func1 as + |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', + |JAR '/path/to/jar2' + """.stripMargin + val e = intercept[AnalysisException] { + sql(sql1) + }.getMessage + assert(e.contains("It is not allowed to define a TEMPORARY function with IF NOT EXISTS")) + } + } + + test("create function with both if not exists and replace") { + withUserDefinedFunction("func1" -> false) { + val sql1 = + """ + |CREATE OR REPLACE FUNCTION IF NOT EXISTS func1 as + |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', + |JAR '/path/to/jar2' + """.stripMargin + val e = intercept[AnalysisException] { + sql(sql1) + }.getMessage + assert(e.contains("CREATE FUNCTION with both IF NOT EXISTS and REPLACE is not allowed")) + } + } + + test("create temporary function by specifying a database") { + val dbName = "mydb" + withDatabase(dbName) { + sql(s"CREATE DATABASE $dbName") + sql(s"USE $dbName") + withUserDefinedFunction("func1" -> true) { + val sql1 = + s""" + |CREATE TEMPORARY FUNCTION $dbName.func1 as + |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', + |JAR '/path/to/jar2' + """.stripMargin + val e = intercept[AnalysisException] { + sql(sql1) + }.getMessage + assert(e.contains(s"Specifying a database in CREATE TEMPORARY FUNCTION " + + s"is not allowed: '$dbName'")) + } + } + } + Seq(true, false).foreach { caseSensitive => test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") { withSQLConf(SQLConf.CASE_SENSITIVE.key -> s"$caseSensitive") { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 2a17849fa8a3..306b38048e3a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -1132,6 +1132,15 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat client.dropFunction(db, name) } + override protected def doAlterFunction( + db: String, funcDefinition: CatalogFunction): Unit = withClient { + requireDbExists(db) + val functionName = funcDefinition.identifier.funcName.toLowerCase(Locale.ROOT) + requireFunctionExists(db, functionName) + val functionIdentifier = funcDefinition.identifier.copy(funcName = functionName) + client.alterFunction(db, funcDefinition.copy(identifier = functionIdentifier)) + } + override protected def doRenameFunction( db: String, oldName: String, From e3e2b5da3671a6c6d152b4de481a8aa3e57a6e42 Mon Sep 17 00:00:00 2001 From: "he.qiao" Date: Wed, 5 Jul 2017 21:13:25 +0800 Subject: [PATCH 042/125] [SPARK-21286][TEST] Modified StorageTabSuite unit test ## What changes were proposed in this pull request? The old unit test not effect ## How was this patch tested? unit test Author: he.qiao Closes #18511 from Geek-He/dev_0703. --- .../scala/org/apache/spark/ui/storage/StorageTabSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala index 66dda382eb65..1cb52593e706 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -74,7 +74,7 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { // Submitting RDDInfos with duplicate IDs does nothing val rddInfo0Cached = new RDDInfo(0, "freedom", 100, StorageLevel.MEMORY_ONLY, Seq(10)) rddInfo0Cached.numCachedPartitions = 1 - val stageInfo0Cached = new StageInfo(0, 0, "0", 100, Seq(rddInfo0), Seq.empty, "details") + val stageInfo0Cached = new StageInfo(0, 0, "0", 100, Seq(rddInfo0Cached), Seq.empty, "details") bus.postToAll(SparkListenerStageSubmitted(stageInfo0Cached)) assert(storageListener._rddInfoMap.size === 4) assert(storageListener.rddInfoList.size === 2) From 960298ee66b9b8a80f84df679ce5b4b3846267f4 Mon Sep 17 00:00:00 2001 From: sadikovi Date: Wed, 5 Jul 2017 14:40:44 +0100 Subject: [PATCH 043/125] [SPARK-20858][DOC][MINOR] Document ListenerBus event queue size ## What changes were proposed in this pull request? This change adds a new configuration option `spark.scheduler.listenerbus.eventqueue.size` to the configuration docs to specify the capacity of the spark listener bus event queue. Default value is 10000. This is doc PR for [SPARK-15703](https://issues.apache.org/jira/browse/SPARK-15703). I added option to the `Scheduling` section, however it might be more related to `Spark UI` section. ## How was this patch tested? Manually verified correct rendering of configuration option. Author: sadikovi Author: Ivan Sadikov Closes #18476 from sadikovi/SPARK-20858. --- docs/configuration.md | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index bd6a1f9e240e..c785a664c67b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -725,7 +725,7 @@ Apart from these, the following properties are also available, and may be useful spark.ui.retainedJobs 1000 - How many jobs the Spark UI and status APIs remember before garbage collecting. + How many jobs the Spark UI and status APIs remember before garbage collecting. This is a target maximum, and fewer elements may be retained in some circumstances. @@ -733,7 +733,7 @@ Apart from these, the following properties are also available, and may be useful spark.ui.retainedStages 1000 - How many stages the Spark UI and status APIs remember before garbage collecting. + How many stages the Spark UI and status APIs remember before garbage collecting. This is a target maximum, and fewer elements may be retained in some circumstances. @@ -741,7 +741,7 @@ Apart from these, the following properties are also available, and may be useful spark.ui.retainedTasks 100000 - How many tasks the Spark UI and status APIs remember before garbage collecting. + How many tasks the Spark UI and status APIs remember before garbage collecting. This is a target maximum, and fewer elements may be retained in some circumstances. @@ -1389,6 +1389,15 @@ Apart from these, the following properties are also available, and may be useful The interval length for the scheduler to revive the worker resource offers to run tasks. + + spark.scheduler.listenerbus.eventqueue.capacity + 10000 + + Capacity for event queue in Spark listener bus, must be greater than 0. Consider increasing + value (e.g. 20000) if listener events are dropped. Increasing this value may result in the + driver using more memory. + + spark.blacklist.enabled @@ -1475,8 +1484,8 @@ Apart from these, the following properties are also available, and may be useful spark.blacklist.application.fetchFailure.enabled false - (Experimental) If set to "true", Spark will blacklist the executor immediately when a fetch - failure happenes. If external shuffle service is enabled, then the whole node will be + (Experimental) If set to "true", Spark will blacklist the executor immediately when a fetch + failure happenes. If external shuffle service is enabled, then the whole node will be blacklisted. From 742da0868534dab3d4d7b7edbe5ba9dc8bf26cc8 Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Wed, 5 Jul 2017 10:59:10 -0700 Subject: [PATCH 044/125] [SPARK-19439][PYSPARK][SQL] PySpark's registerJavaFunction Should Support UDAFs ## What changes were proposed in this pull request? Support register Java UDAFs in PySpark so that user can use Java UDAF in PySpark. Besides that I also add api in `UDFRegistration` ## How was this patch tested? Unit test is added Author: Jeff Zhang Closes #17222 from zjffdu/SPARK-19439. --- python/pyspark/sql/context.py | 23 ++++++++ python/pyspark/sql/tests.py | 10 ++++ .../apache/spark/sql/UDFRegistration.scala | 33 +++++++++-- .../org/apache/spark/sql/JavaUDAFSuite.java | 55 +++++++++++++++++++ .../org/apache/spark/sql}/MyDoubleAvg.java | 2 +- .../org/apache/spark/sql}/MyDoubleSum.java | 8 +-- sql/hive/pom.xml | 7 +++ .../spark/sql/hive/JavaDataFrameSuite.java | 2 +- .../execution/AggregationQuerySuite.scala | 5 +- 9 files changed, 132 insertions(+), 13 deletions(-) create mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/JavaUDAFSuite.java rename sql/{hive/src/test/java/org/apache/spark/sql/hive/aggregate => core/src/test/java/test/org/apache/spark/sql}/MyDoubleAvg.java (99%) rename sql/{hive/src/test/java/org/apache/spark/sql/hive/aggregate => core/src/test/java/test/org/apache/spark/sql}/MyDoubleSum.java (98%) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 426f07cd9410..c44ab247fd3d 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -232,6 +232,23 @@ def registerJavaFunction(self, name, javaClassName, returnType=None): jdt = self.sparkSession._jsparkSession.parseDataType(returnType.json()) self.sparkSession._jsparkSession.udf().registerJava(name, javaClassName, jdt) + @ignore_unicode_prefix + @since(2.3) + def registerJavaUDAF(self, name, javaClassName): + """Register a java UDAF so it can be used in SQL statements. + + :param name: name of the UDAF + :param javaClassName: fully qualified name of java class + + >>> sqlContext.registerJavaUDAF("javaUDAF", + ... "test.org.apache.spark.sql.MyDoubleAvg") + >>> df = sqlContext.createDataFrame([(1, "a"),(2, "b"), (3, "a")],["id", "name"]) + >>> df.registerTempTable("df") + >>> sqlContext.sql("SELECT name, javaUDAF(id) as avg from df group by name").collect() + [Row(name=u'b', avg=102.0), Row(name=u'a', avg=102.0)] + """ + self.sparkSession._jsparkSession.udf().registerJavaUDAF(name, javaClassName) + # TODO(andrew): delete this once we refactor things to take in SparkSession def _inferSchema(self, rdd, samplingRatio=None): """ @@ -551,6 +568,12 @@ def __init__(self, sqlContext): def register(self, name, f, returnType=StringType()): return self.sqlContext.registerFunction(name, f, returnType) + def registerJavaFunction(self, name, javaClassName, returnType=None): + self.sqlContext.registerJavaFunction(name, javaClassName, returnType) + + def registerJavaUDAF(self, name, javaClassName): + self.sqlContext.registerJavaUDAF(name, javaClassName) + register.__doc__ = SQLContext.registerFunction.__doc__ diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 16ba8bd73f40..c0e3b8d13239 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -481,6 +481,16 @@ def test_udf_registration_returns_udf(self): df.select(add_three("id").alias("plus_three")).collect() ) + def test_non_existed_udf(self): + spark = self.spark + self.assertRaisesRegexp(AnalysisException, "Can not load class non_existed_udf", + lambda: spark.udf.registerJavaFunction("udf1", "non_existed_udf")) + + def test_non_existed_udaf(self): + spark = self.spark + self.assertRaisesRegexp(AnalysisException, "Can not load class non_existed_udaf", + lambda: spark.udf.registerJavaUDAF("udaf1", "non_existed_udaf")) + def test_multiLine_json(self): people1 = self.spark.read.json("python/test_support/sql/people.json") people_array = self.spark.read.json("python/test_support/sql/people_array.json", diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index ad01b889429c..8bdc0221888d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql -import java.io.IOException import java.lang.reflect.{ParameterizedType, Type} import scala.reflect.runtime.universe.TypeTag @@ -456,9 +455,9 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends .map(_.asInstanceOf[ParameterizedType]) .filter(e => e.getRawType.isInstanceOf[Class[_]] && e.getRawType.asInstanceOf[Class[_]].getCanonicalName.startsWith("org.apache.spark.sql.api.java.UDF")) if (udfInterfaces.length == 0) { - throw new IOException(s"UDF class ${className} doesn't implement any UDF interface") + throw new AnalysisException(s"UDF class ${className} doesn't implement any UDF interface") } else if (udfInterfaces.length > 1) { - throw new IOException(s"It is invalid to implement multiple UDF interfaces, UDF class ${className}") + throw new AnalysisException(s"It is invalid to implement multiple UDF interfaces, UDF class ${className}") } else { try { val udf = clazz.newInstance() @@ -491,19 +490,41 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends case 21 => register(name, udf.asInstanceOf[UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) case 22 => register(name, udf.asInstanceOf[UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) case 23 => register(name, udf.asInstanceOf[UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) - case n => logError(s"UDF class with ${n} type arguments is not supported ") + case n => + throw new AnalysisException(s"UDF class with ${n} type arguments is not supported.") } } catch { case e @ (_: InstantiationException | _: IllegalArgumentException) => - logError(s"Can not instantiate class ${className}, please make sure it has public non argument constructor") + throw new AnalysisException(s"Can not instantiate class ${className}, please make sure it has public non argument constructor") } } } catch { - case e: ClassNotFoundException => logError(s"Can not load class ${className}, please make sure it is on the classpath") + case e: ClassNotFoundException => throw new AnalysisException(s"Can not load class ${className}, please make sure it is on the classpath") } } + /** + * Register a Java UDAF class using reflection, for use from pyspark + * + * @param name UDAF name + * @param className fully qualified class name of UDAF + */ + private[sql] def registerJavaUDAF(name: String, className: String): Unit = { + try { + val clazz = Utils.classForName(className) + if (!classOf[UserDefinedAggregateFunction].isAssignableFrom(clazz)) { + throw new AnalysisException(s"class $className doesn't implement interface UserDefinedAggregateFunction") + } + val udaf = clazz.newInstance().asInstanceOf[UserDefinedAggregateFunction] + register(name, udaf) + } catch { + case e: ClassNotFoundException => throw new AnalysisException(s"Can not load class ${className}, please make sure it is on the classpath") + case e @ (_: InstantiationException | _: IllegalArgumentException) => + throw new AnalysisException(s"Can not instantiate class ${className}, please make sure it has public non argument constructor") + } + } + /** * Register a user-defined function with 1 arguments. * @since 1.3.0 diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDAFSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDAFSuite.java new file mode 100644 index 000000000000..ddbaa45a483c --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDAFSuite.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql; + +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + + +public class JavaUDAFSuite { + + private transient SparkSession spark; + + @Before + public void setUp() { + spark = SparkSession.builder() + .master("local[*]") + .appName("testing") + .getOrCreate(); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } + + @SuppressWarnings("unchecked") + @Test + public void udf1Test() { + spark.range(1, 10).toDF("value").registerTempTable("df"); + spark.udf().registerJavaUDAF("myDoubleAvg", MyDoubleAvg.class.getName()); + Row result = spark.sql("SELECT myDoubleAvg(value) as my_avg from df").head(); + Assert.assertEquals(105.0, result.getDouble(0), 1.0e-6); + } + +} diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java b/sql/core/src/test/java/test/org/apache/spark/sql/MyDoubleAvg.java similarity index 99% rename from sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java rename to sql/core/src/test/java/test/org/apache/spark/sql/MyDoubleAvg.java index ae0c097c362a..447a71d284fb 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/MyDoubleAvg.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.hive.aggregate; +package test.org.apache.spark.sql; import java.util.ArrayList; import java.util.List; diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java b/sql/core/src/test/java/test/org/apache/spark/sql/MyDoubleSum.java similarity index 98% rename from sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java rename to sql/core/src/test/java/test/org/apache/spark/sql/MyDoubleSum.java index d17fb3e5194f..93d20330c717 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/MyDoubleSum.java @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.spark.sql.hive.aggregate; +package test.org.apache.spark.sql; import java.util.ArrayList; import java.util.List; +import org.apache.spark.sql.Row; import org.apache.spark.sql.expressions.MutableAggregationBuffer; import org.apache.spark.sql.expressions.UserDefinedAggregateFunction; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; /** * An example {@link UserDefinedAggregateFunction} to calculate the sum of a diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 09dcc4055e00..f9462e79a69f 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -57,6 +57,13 @@ spark-sql_${scala.binary.version} ${project.version} + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + test-jar + test + org.apache.spark spark-tags_${scala.binary.version} diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java index aefc9cc77da8..636ce10da373 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java @@ -31,7 +31,7 @@ import org.apache.spark.sql.expressions.UserDefinedAggregateFunction; import static org.apache.spark.sql.functions.*; import org.apache.spark.sql.hive.test.TestHive$; -import org.apache.spark.sql.hive.aggregate.MyDoubleSum; +import test.org.apache.spark.sql.MyDoubleSum; public class JavaDataFrameSuite { private transient SQLContext hc; 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 84f915977bd8..f245a79f805a 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 @@ -20,16 +20,19 @@ package org.apache.spark.sql.hive.execution import scala.collection.JavaConverters._ import scala.util.Random +import test.org.apache.spark.sql.MyDoubleAvg +import test.org.apache.spark.sql.MyDoubleSum + import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.hive.aggregate.{MyDoubleAvg, MyDoubleSum} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ + class ScalaAggregateFunction(schema: StructType) extends UserDefinedAggregateFunction { def inputSchema: StructType = schema From c8e7f445b98fce0b419b26f43dd3a75bf7c7375b Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 5 Jul 2017 11:06:15 -0700 Subject: [PATCH 045/125] [SPARK-21307][SQL] Remove SQLConf parameters from the parser-related classes. ### What changes were proposed in this pull request? This PR is to remove SQLConf parameters from the parser-related classes. ### How was this patch tested? The existing test cases. Author: gatorsmile Closes #18531 from gatorsmile/rmSQLConfParser. --- .../sql/catalyst/catalog/SessionCatalog.scala | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 6 +- .../sql/catalyst/parser/ParseDriver.scala | 8 +- .../parser/ExpressionParserSuite.scala | 167 +++++++++--------- .../spark/sql/execution/SparkSqlParser.scala | 11 +- .../org/apache/spark/sql/functions.scala | 3 +- .../internal/BaseSessionStateBuilder.scala | 2 +- .../sql/internal/VariableSubstitution.scala | 4 +- .../sql/execution/SparkSqlParserSuite.scala | 10 +- .../execution/command/DDLCommandSuite.scala | 4 +- .../internal/VariableSubstitutionSuite.scala | 31 ++-- 11 files changed, 121 insertions(+), 127 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index c40d5f6031a2..336d3d65d0dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -74,7 +74,7 @@ class SessionCatalog( functionRegistry, conf, new Configuration(), - new CatalystSqlParser(conf), + CatalystSqlParser, DummyFunctionResourceLoader) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 8eac3ef2d356..b6a4686bb9ec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -45,11 +45,9 @@ import org.apache.spark.util.random.RandomSampler * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or * TableIdentifier. */ -class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging { +class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { import ParserUtils._ - def this() = this(new SQLConf()) - protected def typedVisit[T](ctx: ParseTree): T = { ctx.accept(this).asInstanceOf[T] } @@ -1457,7 +1455,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Special characters can be escaped by using Hive/C-style escaping. */ private def createString(ctx: StringLiteralContext): String = { - if (conf.escapedStringLiterals) { + if (SQLConf.get.escapedStringLiterals) { ctx.STRING().asScala.map(stringWithoutUnescape).mkString } else { ctx.STRING().asScala.map(string).mkString 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 09598ffe770c..7e1fcfefc64a 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 @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.Origin -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} /** @@ -122,13 +121,8 @@ abstract class AbstractSqlParser extends ParserInterface with Logging { /** * Concrete SQL parser for Catalyst-only SQL statements. */ -class CatalystSqlParser(conf: SQLConf) extends AbstractSqlParser { - val astBuilder = new AstBuilder(conf) -} - -/** For test-only. */ object CatalystSqlParser extends AbstractSqlParser { - val astBuilder = new AstBuilder(new SQLConf()) + val astBuilder = new AstBuilder } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 45f9f72dccc4..ac7325257a15 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -167,12 +167,12 @@ class ExpressionParserSuite extends PlanTest { } test("like expressions with ESCAPED_STRING_LITERALS = true") { - val conf = new SQLConf() - conf.setConfString(SQLConf.ESCAPED_STRING_LITERALS.key, "true") - val parser = new CatalystSqlParser(conf) - assertEqual("a rlike '^\\x20[\\x20-\\x23]+$'", 'a rlike "^\\x20[\\x20-\\x23]+$", parser) - assertEqual("a rlike 'pattern\\\\'", 'a rlike "pattern\\\\", parser) - assertEqual("a rlike 'pattern\\t\\n'", 'a rlike "pattern\\t\\n", parser) + val parser = CatalystSqlParser + withSQLConf(SQLConf.ESCAPED_STRING_LITERALS.key -> "true") { + assertEqual("a rlike '^\\x20[\\x20-\\x23]+$'", 'a rlike "^\\x20[\\x20-\\x23]+$", parser) + assertEqual("a rlike 'pattern\\\\'", 'a rlike "pattern\\\\", parser) + assertEqual("a rlike 'pattern\\t\\n'", 'a rlike "pattern\\t\\n", parser) + } } test("is null expressions") { @@ -435,86 +435,85 @@ class ExpressionParserSuite extends PlanTest { } test("strings") { + val parser = CatalystSqlParser Seq(true, false).foreach { escape => - val conf = new SQLConf() - conf.setConfString(SQLConf.ESCAPED_STRING_LITERALS.key, escape.toString) - val parser = new CatalystSqlParser(conf) - - // tests that have same result whatever the conf is - // Single Strings. - assertEqual("\"hello\"", "hello", parser) - assertEqual("'hello'", "hello", parser) - - // Multi-Strings. - assertEqual("\"hello\" 'world'", "helloworld", parser) - assertEqual("'hello' \" \" 'world'", "hello world", parser) - - // 'LIKE' string literals. Notice that an escaped '%' is the same as an escaped '\' and a - // regular '%'; to get the correct result you need to add another escaped '\'. - // TODO figure out if we shouldn't change the ParseUtils.unescapeSQLString method? - assertEqual("'pattern%'", "pattern%", parser) - assertEqual("'no-pattern\\%'", "no-pattern\\%", parser) - - // tests that have different result regarding the conf - if (escape) { - // When SQLConf.ESCAPED_STRING_LITERALS is enabled, string literal parsing fallbacks to - // Spark 1.6 behavior. - - // 'LIKE' string literals. - assertEqual("'pattern\\\\%'", "pattern\\\\%", parser) - assertEqual("'pattern\\\\\\%'", "pattern\\\\\\%", parser) - - // Escaped characters. - // Unescape string literal "'\\0'" for ASCII NUL (X'00') doesn't work - // when ESCAPED_STRING_LITERALS is enabled. - // It is parsed literally. - assertEqual("'\\0'", "\\0", parser) - - // Note: Single quote follows 1.6 parsing behavior when ESCAPED_STRING_LITERALS is enabled. - val e = intercept[ParseException](parser.parseExpression("'\''")) - assert(e.message.contains("extraneous input '''")) - - // The unescape special characters (e.g., "\\t") for 2.0+ don't work - // when ESCAPED_STRING_LITERALS is enabled. They are parsed literally. - assertEqual("'\\\"'", "\\\"", parser) // Double quote - assertEqual("'\\b'", "\\b", parser) // Backspace - assertEqual("'\\n'", "\\n", parser) // Newline - assertEqual("'\\r'", "\\r", parser) // Carriage return - assertEqual("'\\t'", "\\t", parser) // Tab character - - // The unescape Octals for 2.0+ don't work when ESCAPED_STRING_LITERALS is enabled. - // They are parsed literally. - assertEqual("'\\110\\145\\154\\154\\157\\041'", "\\110\\145\\154\\154\\157\\041", parser) - // The unescape Unicode for 2.0+ doesn't work when ESCAPED_STRING_LITERALS is enabled. - // They are parsed literally. - assertEqual("'\\u0057\\u006F\\u0072\\u006C\\u0064\\u0020\\u003A\\u0029'", - "\\u0057\\u006F\\u0072\\u006C\\u0064\\u0020\\u003A\\u0029", parser) - } else { - // Default behavior - - // 'LIKE' string literals. - assertEqual("'pattern\\\\%'", "pattern\\%", parser) - assertEqual("'pattern\\\\\\%'", "pattern\\\\%", parser) - - // Escaped characters. - // See: http://dev.mysql.com/doc/refman/5.7/en/string-literals.html - assertEqual("'\\0'", "\u0000", parser) // ASCII NUL (X'00') - assertEqual("'\\''", "\'", parser) // Single quote - assertEqual("'\\\"'", "\"", parser) // Double quote - assertEqual("'\\b'", "\b", parser) // Backspace - assertEqual("'\\n'", "\n", parser) // Newline - assertEqual("'\\r'", "\r", parser) // Carriage return - assertEqual("'\\t'", "\t", parser) // Tab character - assertEqual("'\\Z'", "\u001A", parser) // ASCII 26 - CTRL + Z (EOF on windows) - - // Octals - assertEqual("'\\110\\145\\154\\154\\157\\041'", "Hello!", parser) - - // Unicode - assertEqual("'\\u0057\\u006F\\u0072\\u006C\\u0064\\u0020\\u003A\\u0029'", "World :)", - parser) + withSQLConf(SQLConf.ESCAPED_STRING_LITERALS.key -> escape.toString) { + // tests that have same result whatever the conf is + // Single Strings. + assertEqual("\"hello\"", "hello", parser) + assertEqual("'hello'", "hello", parser) + + // Multi-Strings. + assertEqual("\"hello\" 'world'", "helloworld", parser) + assertEqual("'hello' \" \" 'world'", "hello world", parser) + + // 'LIKE' string literals. Notice that an escaped '%' is the same as an escaped '\' and a + // regular '%'; to get the correct result you need to add another escaped '\'. + // TODO figure out if we shouldn't change the ParseUtils.unescapeSQLString method? + assertEqual("'pattern%'", "pattern%", parser) + assertEqual("'no-pattern\\%'", "no-pattern\\%", parser) + + // tests that have different result regarding the conf + if (escape) { + // When SQLConf.ESCAPED_STRING_LITERALS is enabled, string literal parsing fallbacks to + // Spark 1.6 behavior. + + // 'LIKE' string literals. + assertEqual("'pattern\\\\%'", "pattern\\\\%", parser) + assertEqual("'pattern\\\\\\%'", "pattern\\\\\\%", parser) + + // Escaped characters. + // Unescape string literal "'\\0'" for ASCII NUL (X'00') doesn't work + // when ESCAPED_STRING_LITERALS is enabled. + // It is parsed literally. + assertEqual("'\\0'", "\\0", parser) + + // Note: Single quote follows 1.6 parsing behavior when ESCAPED_STRING_LITERALS is + // enabled. + val e = intercept[ParseException](parser.parseExpression("'\''")) + assert(e.message.contains("extraneous input '''")) + + // The unescape special characters (e.g., "\\t") for 2.0+ don't work + // when ESCAPED_STRING_LITERALS is enabled. They are parsed literally. + assertEqual("'\\\"'", "\\\"", parser) // Double quote + assertEqual("'\\b'", "\\b", parser) // Backspace + assertEqual("'\\n'", "\\n", parser) // Newline + assertEqual("'\\r'", "\\r", parser) // Carriage return + assertEqual("'\\t'", "\\t", parser) // Tab character + + // The unescape Octals for 2.0+ don't work when ESCAPED_STRING_LITERALS is enabled. + // They are parsed literally. + assertEqual("'\\110\\145\\154\\154\\157\\041'", "\\110\\145\\154\\154\\157\\041", parser) + // The unescape Unicode for 2.0+ doesn't work when ESCAPED_STRING_LITERALS is enabled. + // They are parsed literally. + assertEqual("'\\u0057\\u006F\\u0072\\u006C\\u0064\\u0020\\u003A\\u0029'", + "\\u0057\\u006F\\u0072\\u006C\\u0064\\u0020\\u003A\\u0029", parser) + } else { + // Default behavior + + // 'LIKE' string literals. + assertEqual("'pattern\\\\%'", "pattern\\%", parser) + assertEqual("'pattern\\\\\\%'", "pattern\\\\%", parser) + + // Escaped characters. + // See: http://dev.mysql.com/doc/refman/5.7/en/string-literals.html + assertEqual("'\\0'", "\u0000", parser) // ASCII NUL (X'00') + assertEqual("'\\''", "\'", parser) // Single quote + assertEqual("'\\\"'", "\"", parser) // Double quote + assertEqual("'\\b'", "\b", parser) // Backspace + assertEqual("'\\n'", "\n", parser) // Newline + assertEqual("'\\r'", "\r", parser) // Carriage return + assertEqual("'\\t'", "\t", parser) // Tab character + assertEqual("'\\Z'", "\u001A", parser) // ASCII 26 - CTRL + Z (EOF on windows) + + // Octals + assertEqual("'\\110\\145\\154\\154\\157\\041'", "Hello!", parser) + + // Unicode + assertEqual("'\\u0057\\u006F\\u0072\\u006C\\u0064\\u0020\\u003A\\u0029'", "World :)", + parser) + } } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 2f8e416e7df1..618d027d8dc0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -39,10 +39,11 @@ import org.apache.spark.sql.types.StructType /** * Concrete parser for Spark SQL statements. */ -class SparkSqlParser(conf: SQLConf) extends AbstractSqlParser { - val astBuilder = new SparkSqlAstBuilder(conf) +class SparkSqlParser extends AbstractSqlParser { - private val substitutor = new VariableSubstitution(conf) + val astBuilder = new SparkSqlAstBuilder + + private val substitutor = new VariableSubstitution protected override def parse[T](command: String)(toResult: SqlBaseParser => T): T = { super.parse(substitutor.substitute(command))(toResult) @@ -52,9 +53,11 @@ class SparkSqlParser(conf: SQLConf) extends AbstractSqlParser { /** * Builder that converts an ANTLR ParseTree into a LogicalPlan/Expression/TableIdentifier. */ -class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { +class SparkSqlAstBuilder extends AstBuilder { import org.apache.spark.sql.catalyst.parser.ParserUtils._ + private def conf: SQLConf = SQLConf.get + /** * Create a [[SetCommand]] logical plan. * 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 839cbf42024e..3c67960d13e0 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 @@ -32,7 +32,6 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical.{HintInfo, ResolvedHint} import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.expressions.UserDefinedFunction -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -1276,7 +1275,7 @@ object functions { */ def expr(expr: String): Column = { val parser = SparkSession.getActiveSession.map(_.sessionState.sqlParser).getOrElse { - new SparkSqlParser(new SQLConf) + new SparkSqlParser } Column(parser.parseExpression(expr)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 2532b2ddb72d..9d0148117fad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -114,7 +114,7 @@ abstract class BaseSessionStateBuilder( * Note: this depends on the `conf` field. */ protected lazy val sqlParser: ParserInterface = { - extensions.buildParser(session, new SparkSqlParser(conf)) + extensions.buildParser(session, new SparkSqlParser) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala index 4e7c813be992..2b9c574aaaf0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala @@ -25,7 +25,9 @@ import org.apache.spark.internal.config._ * * Variable substitution is controlled by `SQLConf.variableSubstituteEnabled`. */ -class VariableSubstitution(conf: SQLConf) { +class VariableSubstitution { + + private def conf = SQLConf.get private val provider = new ConfigProvider { override def get(key: String): Option[String] = Option(conf.getConfString(key, "")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index d238c76fbeef..2e29fa43f73d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -37,8 +37,7 @@ import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType */ class SparkSqlParserSuite extends AnalysisTest { - val newConf = new SQLConf - private lazy val parser = new SparkSqlParser(newConf) + private lazy val parser = new SparkSqlParser /** * Normalizes plans: @@ -285,6 +284,7 @@ class SparkSqlParserSuite extends AnalysisTest { } test("query organization") { + val conf = SQLConf.get // Test all valid combinations of order by/sort by/distribute by/cluster by/limit/windows val baseSql = "select * from t" val basePlan = @@ -293,20 +293,20 @@ class SparkSqlParserSuite extends AnalysisTest { assertEqual(s"$baseSql distribute by a, b", RepartitionByExpression(UnresolvedAttribute("a") :: UnresolvedAttribute("b") :: Nil, basePlan, - numPartitions = newConf.numShufflePartitions)) + numPartitions = conf.numShufflePartitions)) assertEqual(s"$baseSql distribute by a sort by b", Sort(SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, global = false, RepartitionByExpression(UnresolvedAttribute("a") :: Nil, basePlan, - numPartitions = newConf.numShufflePartitions))) + numPartitions = conf.numShufflePartitions))) assertEqual(s"$baseSql cluster by a, b", Sort(SortOrder(UnresolvedAttribute("a"), Ascending) :: SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, global = false, RepartitionByExpression(UnresolvedAttribute("a") :: UnresolvedAttribute("b") :: Nil, basePlan, - numPartitions = newConf.numShufflePartitions))) + numPartitions = conf.numShufflePartitions))) } test("pipeline concatenation") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 5643c58d9f84..750574830381 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -29,13 +29,13 @@ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.execution.datasources.CreateTable -import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} +import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} // TODO: merge this with DDLSuite (SPARK-14441) class DDLCommandSuite extends PlanTest { - private lazy val parser = new SparkSqlParser(new SQLConf) + private lazy val parser = new SparkSqlParser private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = { val e = intercept[ParseException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/VariableSubstitutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/VariableSubstitutionSuite.scala index d5a946aeaac3..c5e5b70e2133 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/VariableSubstitutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/VariableSubstitutionSuite.scala @@ -18,12 +18,11 @@ package org.apache.spark.sql.internal import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.plans.PlanTest -class VariableSubstitutionSuite extends SparkFunSuite { +class VariableSubstitutionSuite extends SparkFunSuite with PlanTest { - private lazy val conf = new SQLConf - private lazy val sub = new VariableSubstitution(conf) + private lazy val sub = new VariableSubstitution test("system property") { System.setProperty("varSubSuite.var", "abcd") @@ -35,26 +34,26 @@ class VariableSubstitutionSuite extends SparkFunSuite { } test("Spark configuration variable") { - conf.setConfString("some-random-string-abcd", "1234abcd") - assert(sub.substitute("${hiveconf:some-random-string-abcd}") == "1234abcd") - assert(sub.substitute("${sparkconf:some-random-string-abcd}") == "1234abcd") - assert(sub.substitute("${spark:some-random-string-abcd}") == "1234abcd") - assert(sub.substitute("${some-random-string-abcd}") == "1234abcd") + withSQLConf("some-random-string-abcd" -> "1234abcd") { + assert(sub.substitute("${hiveconf:some-random-string-abcd}") == "1234abcd") + assert(sub.substitute("${sparkconf:some-random-string-abcd}") == "1234abcd") + assert(sub.substitute("${spark:some-random-string-abcd}") == "1234abcd") + assert(sub.substitute("${some-random-string-abcd}") == "1234abcd") + } } test("multiple substitutes") { val q = "select ${bar} ${foo} ${doo} this is great" - conf.setConfString("bar", "1") - conf.setConfString("foo", "2") - conf.setConfString("doo", "3") - assert(sub.substitute(q) == "select 1 2 3 this is great") + withSQLConf("bar" -> "1", "foo" -> "2", "doo" -> "3") { + assert(sub.substitute(q) == "select 1 2 3 this is great") + } } test("test nested substitutes") { val q = "select ${bar} ${foo} this is great" - conf.setConfString("bar", "1") - conf.setConfString("foo", "${bar}") - assert(sub.substitute(q) == "select 1 1 this is great") + withSQLConf("bar" -> "1", "foo" -> "${bar}") { + assert(sub.substitute(q) == "select 1 1 this is great") + } } } From c8d0aba198c0f593c2b6b656c23b3d0fb7ea98a2 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 5 Jul 2017 16:33:23 -0700 Subject: [PATCH 046/125] [SPARK-21278][PYSPARK] Upgrade to Py4J 0.10.6 ## What changes were proposed in this pull request? This PR aims to bump Py4J in order to fix the following float/double bug. Py4J 0.10.5 fixes this (https://github.com/bartdag/py4j/issues/272) and the latest Py4J is 0.10.6. **BEFORE** ``` >>> df = spark.range(1) >>> df.select(df['id'] + 17.133574204226083).show() +--------------------+ |(id + 17.1335742042)| +--------------------+ | 17.1335742042| +--------------------+ ``` **AFTER** ``` >>> df = spark.range(1) >>> df.select(df['id'] + 17.133574204226083).show() +-------------------------+ |(id + 17.133574204226083)| +-------------------------+ | 17.133574204226083| +-------------------------+ ``` ## How was this patch tested? Manual. Author: Dongjoon Hyun Closes #18546 from dongjoon-hyun/SPARK-21278. --- 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.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- python/README.md | 2 +- python/docs/Makefile | 2 +- python/lib/py4j-0.10.4-src.zip | Bin 74096 -> 0 bytes python/lib/py4j-0.10.6-src.zip | Bin 0 -> 80352 bytes python/setup.py | 2 +- .../org/apache/spark/deploy/yarn/Client.scala | 2 +- .../spark/deploy/yarn/YarnClusterSuite.scala | 2 +- sbin/spark-config.sh | 2 +- 15 files changed, 13 insertions(+), 13 deletions(-) delete mode 100644 python/lib/py4j-0.10.4-src.zip create mode 100644 python/lib/py4j-0.10.6-src.zip diff --git a/LICENSE b/LICENSE index 66a2e8f13295..39fe0dc46238 100644 --- a/LICENSE +++ b/LICENSE @@ -263,7 +263,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.10.4 - http://py4j.sourceforge.net/) + (The New BSD License) Py4J (net.sf.py4j:py4j:0.10.6 - 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 98387c2ec5b8..d3b512eeb120 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -57,7 +57,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.10.4-src.zip:$PYTHONPATH" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.6-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 f211c0873ad2..46d4d5c883cf 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.10.4-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.6-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 326dde4f274b..91ee94147149 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -335,7 +335,7 @@ net.sf.py4j py4j - 0.10.4 + 0.10.6 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 c4e55b5e8902..92e228a9dd10 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.10.4-src.zip").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.10.6-src.zip").mkString(File.separator) } pythonPath ++= SparkContext.jarOfObject(this) pythonPath.mkString(File.pathSeparator) diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 9287bd47cf11..c1325318d52f 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -156,7 +156,7 @@ parquet-jackson-1.8.2.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.4.jar +py4j-0.10.6.jar pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 9127413ab6c2..ac5abd21807b 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -157,7 +157,7 @@ parquet-jackson-1.8.2.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.4.jar +py4j-0.10.6.jar pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/python/README.md b/python/README.md index 0a5c8010b848..84ec88141cb0 100644 --- a/python/README.md +++ b/python/README.md @@ -29,4 +29,4 @@ The Python packaging for Spark is not intended to replace all of the other use c ## Python Requirements -At its core PySpark depends on Py4J (currently version 0.10.4), but additional sub-packages have their own requirements (including numpy and pandas). \ No newline at end of file +At its core PySpark depends on Py4J (currently version 0.10.6), but additional sub-packages have their own requirements (including numpy and pandas). diff --git a/python/docs/Makefile b/python/docs/Makefile index 5e4cfb8ab6fe..09898f29950e 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.10.4-src.zip) +export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.6-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.10.4-src.zip b/python/lib/py4j-0.10.4-src.zip deleted file mode 100644 index 8c3829e328726df4dfad7b848d6daaed03495760..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 74096 zcmY(qV~{R9&@DQ)?b&1Qv2EM7ZQI&o;~CqwZQHi(`+n!1b5Gs;NK#2xex$0qv)1Y; zNP~i*0sT))&s3EBUz7jcf&Vu;c(Pd0EBtR0C?LvzO%f{;IB_}u?IGX*0U`Y#6C*=o zYX^HL7di*eVGZ5NB?ctl+ghj(2W^e$w>;vSJ@$f#yuDzFf7e&r7MV+=%bWhUq_X^r zH#f`P&42VV$1|9dGMLYINZV=CIs65z8arM2I~)(ioa!ao@ltTF#5lo^S4-PblK zy-)^8!w`v0o|QoxZ99dFDMc75zp?k3fo2|d1Z5ZRo8&x)3LBju_h)PcAkPz;!weMI z{)}eE=T9UMfkS~x(jv?~6%`-Wl>*`w0Zcx<@i_P*VX4?uGh!yOxFp444%{297-k?m z73R}T0KhRE(Gd<_T1K7%!Q$a?xK0?PJ!+t3Os~N@G}+=JO!)L>@SVh~N)E#5XjaVA zRRDFHHyv^zAeZ2SfNR`pgV!L@{#q3~20qP;IFg(Lug1yL$V^9<3?>ECLu##>*(a?(5%JZX?j+I~RO3kc(>$>z<^^d3qMZcn#>Yyk>3K3x@ z41k6#Wfu>{KQP9Ch!`waWhjP4OnHeDZz?j5zfmZ*-&VpVG=IBJk}bN_0U->h(1&=Y zp~`^s;&XijeW46KegJUco0?Fqo%t~6*a47k(LUcZ?bpCjm|U>zb{nnFC_C=*zfTp~ z{PT<6x^kEuR+R|~$-&ES&H*Z5d-0<0iUrH`F1V~G9oquImm??3f(8A$X4u&kMt+q_ zM3ZcJW_!RqWM%)WpjL8jM{N$}ufkEtuO{oyk}uXA%2x4V&5?AxV8#2I!e8miKmH@m z?=@RFwUHG-LGB1rUTKaoJ^I&ya=SIlLYGdM^Y4Ah{#Dxc8KFh3uJhJMq&_YnLD@VdMa{Ec{;W{F%n!HV&j7gCSgph=&y;u>)}hhC|R z7tx;^DWaz6-v*>G65LUz(D0J_hWhps8Wk@_6) z9jV8Ps*QfoVZ)cr>*8}*4hem=l%?0QDO~1rkQ+kLzWVKlhV1*h0 z-SccbgI_x%J&EHeXW-iq0Xa{m%l3p)( zg(;{&Kw2i{#Ud&wy`v|b8$9xax-=MFPvZNXggoBAovLZJkK-%pv>f=jQgf3?&T7=+S9>@qnQ zRmjEhp3~G66(z%1Q*ip6#&qJ5z$ur6fMC1>*D0cEnph38HZ}}sekyF0RZ0eSp+3lr zG=`V;jB&z;OBL;T8F0{4cO@D?D$c@8Q8$N9gkA(YLMb-g6MbZo4^$dsg#gubMz7Sj z?9mGRIwrZpb>e@WkswsdjMJbsttq8VbF0)T7+oz&Y-4JmJ2oyP0{Bod)5?A!W zN6+KN@>p+TiPB1m6j&4lm=f0{@_bgsMv3m_$n*1lpBP~P0J*y|_V%v*tE=qHRu$5I z>Arrx=zhM=-`|K|TCIHCS$@2*=K4BvwY`Cvn5E{6v$gayrR+{+Ly}nR-QlJw_H0Is z3k50*n`oXXqn;O{WWQ`h)jR8nKVGl>X^RD8(E)yX{X-MckeW(0{ceMy2Dm!;QV-8K zRcu>*utsrGu)q+hp>c{@(enlj0>;Rp?gdq5a4B?UQ-mn=jOS6}Y(p51I4NZ1ax{bP zOU#)HX$G6S=<{Zvx(eKG_bsMq^YktLX99?OzMG)vatO}` z=LmeO)`zKY3PMpisuG*WtjWn)^FkJRWF+>akWin*E`W=!ZEus_05F=5NBubT*qXjp z!i3)XwB5RTy@5E?(hQ_3OV`)K*oe0XV=Av|hn;VFLx}sndTN?YW<)!NqgNlefECuK z&e)1IyPH9zZXZr!HFV@&JSs@sV2ngllwiY!Mi)JJcwYT(_@`htK0DaRm>^P-UIe5# zDwN7f=lYn7N8DxK5h}jo8!)7j zqA?>t6;gG}u^s5J(K_Az8DZc3pPwO)K>_hFA`hmhwY)0i!&|OTd6yj7DSA=l#Ho1} zOJOvsywZXd@ViS1YO3eM48DK32G8KPwYHJreW*YyT1BPF6veW09BO0;p0&(5Ln(1KwEj|rg_3tHSjY=5Pw#6?6U0-UxuaSKE+$Dzny%=wb1>#>hla@`@$lPdb z;l6+++M)?hkm_5S7igBrBVjS$>|T^^A)2KJ+yBIRh9^-)pqjN|B4I#c#gx(V(iN23GQ&d{yID$&I*SXv+uZHy%@ zZwj#hjuzu!opRqD2?frtnQuc)D2JL|n+(wobld!^V@h11N4tzz2qNuO7#ImZx^M;r zJE;$IC8f=}SJVCUhN)m5?5U6P#05(jy+jORX*PZ)rap;)NaIx;DqS48a`5;su(33qP z(C-Ndau(t96nS4zW16r5$8(~+B^KwUGgM1`HL6E^(ikk)X%C9B4b1c@)2)Pv%j~3{ zQxe=XMuJwFhdTVXpd&-&Vf2uvrf}4bb0aJJRgEO?je(Zzs$rGiq>g^9>TZ|*%2coV zMS@6NLHXCMMtvSgM~)l%;?ov13z1@oJXE(bk3d~ zjVD5q%4SS4-yY#!$P1)m#lizf!L2AwMEsg)t+?rIZmof9}g+jCpRqSTQI27ks>hF21Y$50fb&D0p*5q$>)lMrlVy5nXva^{R= zA2R7>%cNS;#AtA=y^D+YD^SCH_>c_2n<*0Th9{j33SHzkda@@GaAS|iD`0O0BV-%3 z>nu|T+EgH;nn8cv9I4NAwUNBZOIl95l^5!PAB!u$+yt$?nj<_ZY#-cQt0}hwCcU!i za&YB9GUxJ>_^oUJu@Tu(e4AZ#b^qR)!2C1%?)u2R$TQUfw?LH|XB4|{OoH}{!wGDC z%j|FuNwFD`;uf;(gdlxz=2>8{#e>Dw)PmZD$L&KSxDoLaE` z)rzSEffj;y3Y&y5y_yg5D;(J<=h>S^&fvDdj>M_EiT!cCg%Ely+^VdJ$|eTvbX#Bh zx;On6DE%SC>4C=*pv7v_z}1JUvz+W+286fuVR6$%pbiG%zx%^#FG#(xVYbF#FbDSF znxIQhZXLx_?_F5VQBb3x22$8+!8I>Q-?Z!N64B=i34EpYAT+>lzbK%d14{6~Up#N~ zMkOKbCfzjlO1uPaXyJf;75(vezt#SnzwCXS8a5n&g8&pUxzB&j-3|V-vt?KWXRdZz z_vz|j!wZOg*n(v}i#|}QJ70??w-7$96SeZ4{~N{dPUZ4yvq9XrfjW)dK<*b-plz_kN75`RWl39lW6qTS~CUa7W0 z55eDBgJ3Ryb@m80E0mb^sgH`-TasMw);Itf*FN4$-a>Dv`+er}dAfx>9dl?$skzHL z!G#Sl%1C7?CMVaPV9um{_!!2xrbp<1%_~W|dHC#8t#d5PW*s}lZwA3;sC%@ve{Wa2 zq<-IF3NwpE$<5&N@>L@=NaLo3q8tMnleoKHk+#Z?hxBF;Jc^_ zaZ*~q!NP?%`jWFK;%mDap(_iQlH^*JtUKXvxL#TE+=FBD`AP{ZTujVq(YA9mKTN`P zs?U`$dFRV_#BPn3tBhJrA*tETuhg&hjw=rj&85vfHxwf$t~2A&5umj^;7W_v{0c>m zm{OCOWDa9E_CecHo1TuuP)3q;r!(6W=Dk`f+~|8VcU-UH!_iTA&pNU(Z8}~}EPKS< z*Q?be=XWaR!i!Hv)Fvpb=yZA44k_6WFV8d1`0VT#s7)KK_r9gFb534`VG)|cj3c6WB*=cB)OJLr66UAbhYgf zLEEs>`vc~@*MrNn>n~uX7j((JMFvc?Wv#NoPHQRl!W2U zFE(O3P8S_B9ca#wI?F2K* zbe^lV6!uWP4AS6ago29*mlp>%+~JX{i73lIEN!+?*|*X;P7a|(QD@ThVM}I606O^X1xFLIGx>3 zrz2D8F*14rJF+mqvU)tHA{uY0?%GYizW}7WV3Tje^B29#m6R7op{Kohny#Lc^3W5MAWr=z<0Lq`!puV%Z)ZYxW;Noa ze68c`cnd(E>q|G2bNL$Q?V8Enc}UV$j4h zqq3p4NxTto^`}@!o|U7gqZHgmkeRamwHRdW(;JgngzRA0b+r80oH2I;t!4KjI8*~T zF|u)g9+s<#r;+k;cfW%r!x^=;r30p8QKVYwyebFbeR?%Fe#ag@-j~P6s;>%??Pq5C z#)LQb5YbDGw`<~1csP5<379~S`H(|&b{}#)gm)DR-o{2VRq2uvO;b6O!M(?(7zeAE z8VD(cwRiu7A9w<*|82aNhRqZj?Kk@2zQ50fisE{E5s|<;Uw?|WFNJ_unB(dbhRAn! z;0kwvSjO!eH<)TqWx(U4fvU!rRqv!Pg>fq6 zR5fSr;FLq1wlLJ_Ir4`@w%j&$Y3oXp4-t;Ze02B?GXPpvTODb8#>RtWzWBVqyQ{rB zQ=q?dZ-Eua*h`u3uMKQ=NXIk&MA7H^@ zthG>VS0%h}YqI@}-BC?)Jha(&cBjkMGaZE6ie16Y+M~=sK*)$Rpi*mxHG)7axd9aw zHk)W5<5@$xxs2*b3mwD!WBiWt_$0CpkHypeA zyhjDze{7R$bL?jgLbIr2tJ2B0A1wEyuy1kILaq8%Y3qP!q%P-iDKV=>UfG*N&o(W# z#eXSPp`UyL#k_qf zW^`@;QI0sP(HgU%kC*&XjOCezb8jowfgs)6rl3~Lz*%>*_<^4>SXd(u-{Sr0?P>S* z4tD;Y<5bz*ya6ppbcc-q^YL5Sil4c&LNgHgr4b=Rx|&9xypEpE5b zm?CtdzM4yj*m_w3G-ZzQX&ku+>7t%0_x9Giqo^6aG=m+`LI*@67QA z`~Ro$?muHazCnP1jG+HhW&S@J&)(L-@PG7}Zq@17MFymvQ!24-=>}@#3*r!JFAbu- zfDnN{jslfqk`&eM%G}o6n_3WQih2@yD3W8+5ubxAy$P(p#V_nW?Jgt zNOLg3!6Tux^o$q9)s0gY?a-;4a@qH7V8OcpsYNjqIwxovTqHyysq8_dw|1SaJ}dBj zBZS&QQ=lhD^ofsU+w6v`nG*ZTF9Ek#c=$eWD9>3CS{4MOmm{@w{sF*>pC_pO{se`_ zCPy$gevK{ev^bHhTp!m<4JDS-vS&}`5EM`P9YZmp(WPZZR3!Yu6~c@~=WCD9!kD@p zHn&oVBJOZ?r#A<2^Qm0S2-k!nu8CB=mtf3IKH@g@KsO_Fk=KfJF9DMi+>7(#Ka!;+ z!T2OF^Yk%>C1+ercu;AI-%93tLM&(_tcB}?+hr~*{Y67DC&K|39J=MqJ!2+Ux2WtF z9;3H&o*8TOujT=mzk7c*-dSJTl73N_v>j{*|bt~-C5@f!)lqjYzLnz0w zD(k{aCKkmj3U^7$Ju7#k6U<%qbzb7lWz4f@3;Y)fN%)i#N}0as154zWN`g-V2K1FH z?Ie`P0k}6T{SuR}sDF16o38X`v`aGb%`&yaW37Wnh$Blv%^S;9y+?IJn7VU;{G zP3Os@N6SCz+WoCX6o1_>5(g_-%loSgw1RuvzFjdtim7BUZ{E~i^&A}QbBA3O$GUIm z9*{#x;bbh23 zES1x}kki`J=PQtlb>Wlv12xB?5R&Im=Jp_>N^cBaeN+f@)IN@vWQ zZ<504AzqZS{OCRzDIgaxXdtCAN%SLCy-g>TXJv{eqlBi<2!n)eb3eVwAsyE+w=QP( z7-*_iG;oHVUBe+e94S??TowSR=UlU#aNd=uZU|9urUbO_p`}AN0<|znxnMjRBzVYE zhk|q}d5^Nwt2iRlZ%%&ZE()>!ZTx2KXC;?RR5<6p{Z3VFJ2og$%dM|_qy_$<92~tn z4_K$zi|uGupdlx=&mjV-+IRrRHNiv(lcQduggt14WMDX?Dp5&e$IY;5t3Gcs5USE} zWRyn@7i?D=LW=m*P^BSXD;-jk_{}$S%cD6{ta`>|7#*sZ7->&q zDKi36Fw=QKTy2R!qPa+GGDRk5WP85t(fsZ^oEz59T(Ggxv(Ed-d+X0X*>gmxB7$Vo z9`JN@I|7gPyW=AjITdS_t1L=j<`seD6jFY5o3Dmc%au9Jlg}B;fKhndGHT;oH!KZ!92oZ*{B?XYe@a^xnuJ!> z;k6>Pixz&*@~VB*S^2TREj&pLN|2eUytb${LFk*D-m-EdLwSV83#|qz*?-WLkHbsr zfJ*jeowK{$Bb}sUrzN2co8cuL`;N&&Z{Fn`f&VXFp9fC*i0$)TX8HoIodaXJlH~p| z`^%#|Phq~bmUlZzGvgT+C0~cKbw#7L|nursAQUdyP zK+G;$Z0lLQ9HuONI;c+3s3oFSJK$LU|2qBuh+MbzX_VjEaky{ti&!pd|U z^_Wzz{R9^zp)$<~Pd@1ELZFM_=jlP)%CdR$ zdc~4QGen70Y5GtCoWut(0v%C^n^9zd)ce8C*At$PS1O1X>3H^^0xV(dST9pMZ^W%# z(nu4w8i>OD7ccjK)AchS>#q{>$d8EJkOdXF(*^xN)23B(EXsMQ#Xk~@n4wB_{Cg{V z|Jr4)JIuL6ya>WC((cs!?7zctKPfR^0{2KIl02vYcHN?A5ibCg*&~PKFfj~+$$6+C zH7dUf{(hVa9-Y#;WmM`((@JH`vL*CaHk}gT56APV^IJX%#6xXp%t%Ah^AWs=9j-F5 z>~<=0N$P|}CVj8xF8*j^du}RgLg*WN@V|sE(BJ0TFBVxn*S*eprWC)#H)*xtFi!7p zYb$lj1!NNadC0XhjHfQ~wJw^rrxkr1TfB+KdGDB(!Cd)c8&aZusgfNTp$m~#LPfOd zsu*9ukR_ZuwEp=lxT~hVrvc`qz27y3q>YQg6*GUPhqH)MD=CU#bQdnaPG4aj$F`a@1p%(V%)Z>=P<^2*q6SbRc)!ZPEDC!F4>FdQltnIC z@@D5>GXg#7`x?sMoOEL61MMj=7t2V_q-XRI8kC}n5Iu}1yZ8}j3?a@}If;k3w(eA9 z=mwOLba2I(aTMM0A(lzH5dkHO6DUMK7^PX~WVllNgb(O;76ByLt!_uGv50Vczbg3- zT<6m%k+o+^;qI|q$$3=UkA)fN9<)J?7?2naxDux>JmZH^WjGoIQT9g;D*o!YNJqdJ z?ir0lCyuq7CR`XkFDuR`tnd=m@Q9XCL!%UwI*OEI?%X-O9jbo~=tpekuIV+fLb*j9 zQ`k|4V>b3!rp{y;k_6CH2r5zfL0Y`KF{e50ztKZcV!ZY43pXcE^Lqovo*U4!y?=gU zmu~Sn#Y^>0d7`IlBB$X$fh}5N1suZ~GL3AC{Uk4?ThAo}ziLk~efvi-@YoCcVRE1u z(;SO)$?ld7O>7%TyT%nsune&c7AlD)yy|X$Ju-$CA9&!h_>vtU6b{gUnj3Uj1ZU#< z)71w|H2ZMl(|x$$gsW0UVDfCLo-*NPbcVo)Lvkg8v~ozvZ{*?N@=&@5w2a<1Ei5}l z59b6bb!+y6byiMlQ_k}`hiWda@cU^ZQVQ1Vr8rtzNlBCEwa(5e_UzA-HOQH&E~=?z zj%a%lZ0^+rmcd7Xgk3wa`;t`_mC`;0i`X1A#XhohR(R8PjKjjgJ>^)y*|ybA58+`! z^uz*RM!WqHhDN{rH*~K^%>6EqS|5S$h6(L-S|w()rV*i#6vpj>o2Xljh)w)Mp8TuM z*cvK7`4EW1*S8+TfDm+91?RU|IOk2x$$U4B)TrCdl)UtQpN!tZ+0PIym%*wZPpUV~ zLi8%=S{Pvx8E;GOv$|UkxbK_e^Aj@{Y^||}uHlK!jvUrzhxos#b;l2RxG zU)(}5Q0g9aC8v2=z5Zifna*m65d?|c*$l$Pzg4%=!K|(*tlygoa7yj$4B$$(RI~u< z)r#qhU4HAu|AITMfbAKAzf)(AIvR6y1t0v%Hj(z%9=sXk5N}b1wp9ASk5@W|9)_w6 zm3`BSi0)@h?>Z+zZaQwPg8#ZD?_I}R4S@8mgEYh0V`nq+4&p+Y_e+C@bFK!)qs4L)#T;VuK1^H^O$=VXOa z`AxQfK34HAhP7m_bIL^<*NzKYTV$@ha3P+toYn-lXivT+(9IN(Q0XDeh|cPdcSqCJ2wv9=?`H{Lyr4XFS&}HW+Tyh*I>hpgN~O&8S}nnGcs)) zU3e$SenoagtkZ3q3qw$yQ*?$hIY_UaN=j+rH+mRja1CBDfJ-6GV2~rPNjBh zHRUV9)Q2Jd!xV($lzNeilCQ5@R0e)5L2~=l6;cTv$;ru3^u7hLG96bdEbtz-j zvXPVCq&)%$9?ev>JO*8N^JkEktgW}T>wA*G-Cj9Sn~+WRXK!%uzgTTH5%5F-RZ+*q z24EW#8#`Sr3YM`+1x0c&b0Zcl0M7V7aw@U$Sh>l!g3%B!8P~Qq{A>t%0mWa-Pbxgl zRdYBFp|+?iHBP`qcd~5fMLCS4VHEM2lh91FOz|NGQ{CVRPFGd14yp(nQ2etd-?=;J z0Um`7wQcBd6ZY4Vf{pa9_-1PM(<}jgJb=&vje03bpqej`S+e&=tHuYnby^T_?2^Lq zpp>Kk>VCec*4h&yAA&WFe(Krmcki;brVT#NbZT`z$gby9T6v8L+2#IJ-=BXu^so;K z=4{6zpW~aM4`?UI4kuhIW)0nLRHZK7I_iuuE(9`5o2;Lq2SO;GfU5*1BFyH_@_lN; zrA1kNe35Eva>)@z2fWRk!qDc1xeY5XtLHlQwG279;(YJea$)p?cd zf%jhIj|L48^(U#Z*>!>I<%faqKY{M!&b0^EOTi>sxY@A=A!prwMh$y{>F*D z?8bE+jHU^_8-w6nS=|oBA6u2&12Y0{Ks6)?NpqdIPvx*RY9;1q9B9n)adQ8K;Oe2g zVy$4Qtj@hJebNrgH!i4Jm1qU4oTlXLOv-mFycN!A;eMC?JdIkD|6EMOCd{nbF=R?7 zZfe(SS~qS=PgYuQXG&k3DnuIoS3BNL72eLz*w2v%!&V*(Ig__Rue0UXd)LpS=eIu_ zRb9>RXl(5g%I5Wr`c369EQw1a{7WzhwCyZ=L^GuP*up1(nOKb`fL-p_uT{B@7p5hh?NtgK!@!Xqg=|I!i8s7$?IS*xsl0$#Gaw`p31q$<)&l zGjf_{Hx`S$sJ*92<4|Ivaht=Mma3#cTuPk-n;kcnj4~p9)cNv!M-wmQTc+COSpfgv zP4no|LGc=WyTxX_3aV8SGql;xUcFoickT+I);~-|tt0=UcORFZkE{7pX56-GA3vcn z)%uKyP0nVOo*ey~i;>a6&H>cbCx$MczBL@wlZIdKU#B|d;*au?j+65TdQpBQ57yKh zR9_ujsSf;QcF49?5t3kn#$)*%rAk06IAg3|?AyVbEsz(+JlOLIUMtV?E%-Gt#czwB zR!ra~W%dP9GBY}Sv7xEVT>Z^k#q_}2$C5;sGvbi$i-+IN=1)i*;Ih>k2Y_DNi!*o| zXme)-r)R}Ev6JO$^ypS4s>MH{SIIh_oS=e+QAltamy79Wnv(8alL4u+v4I)yiq#R# z0@d03c7d~fJyL0gkCr06*II7XO0pLK`;S%2GbNWDJ*?&8B%9Km;^x`?=``)&Oe^+1 z;VG;&eA%@N>bVMU9t{1;)7ePMK2TfpgwvDanSTsw164Q)wknp=e~~B7h`UqLz7rb@ zO<#!JWlLR%m$<4s8=$By`P@`rTg%Vm{k9D;RKV!Z6y(iJxg^B&5l)IMViPZ#H?bI@ z2ekb+8mBNaC!Hyw${k|{VH44vZ(0^EZ~VFBlW%lZBZ#QR13$Qq;C3M%udtW;0G`Vc z*I43Rdsv*)vU2(96e1f%qQj1vNM<{xE(SGZ^q>E9pXIvl==pvq``2N?ZolY*4aZTUE{MvR3;>RYU_a8#TrP0BkESL5<`lC`_Hy%i;wo(WYIMLJbcT)9<=oldSvD(e1p`}BqPbRt}L&bF-SD}&O{jWg|o?7(lu#u{e@*=gS zN@Wop`SP`GeLBd)wm@doCKyXioe(hn>2ar#A3qo#;w7;@`XX2!A3gc_tN8dUfbZL~ zwl%FbU;SC{fhwSz>SXxZ_kpLE4#RVOuk5+(QIk!(cEvTpRMNAv>H@uw&M-OJM(;pl zCQC6c0cmvPM*LF#I(!dEXgpYijdXPt2DcnvcJmGSgwC`%2l! zGC}s&^_P!y^Ly2vadwbH%zT@xDkbOF+-56G>#@kpN6p1cf-oB7Z@`0>;bX886UzN0 z5yi&er7afGdTAcqT^TR2tNiay(Zc*PYGtYg=>vPY^mQ-`kI3yGmMeaCDBCBpKei~sXG8h#UiMU-VUZdv1B;H`x~XdOgWbsK&q^N& zF4zT(PJC0xgK00Hz1pZVk4$)Te{tfoF`oq@=u&nEc?UeTrV@Z>Gx@PE=l~p4Kw9SOVVKjE0Ky~JI-?h*-cjKJZ>L8 zf9h60pPtQg@pH^ENBwz%wD>(f773pLguZTfAIGL{cVF-C^TXggH~gPFaf_vQz2?53 zFIP`rD0;rP;k&;-uh090bpF0xi8(pBKR<|ek%W|UgM)>QlUEOK%k$^fWp@vIcW0}m z^tpsN#@tF$+MGNd?+=T?gS%#!;-~*DLN4b|AbOkozxCTUMDG&hnueFs>l5JX5#$1p z7OLq#bbcPbSYKMH)PmW+KMT&|5`TDq8Vmd{T<86{L)Pu0aL5TGt@?c^tkssCpsVcl z&&(J^Mn-mlcRLyH;B!mG9D#Od(vmOZga$w$lUpd1W@C1DcIgn!=JqFTVaKj8=0#<7_&*&NY4~Z> z^83iS5^sL=O*x;(RSBXl1Rt<%b44(R9d>WzSZpxf%D98(LeYO` zda1Ei#%;2OfS|p)x>Z1aa!ONBel&~`59qn+s$ZJE+#ulCLo^Hv7Z-{))`-TGVvzTt zl|(KLm|?FZCx?oHi&}lA8kFz`xq5<{2I?uq+oOI=P_EDu1+&2ajQ4YYK!UVgikT7`vTVl49^`)kWolJwaf}DVxkzqM3 zg$cv7jtM!MHn>eOjtiQN`2|5)*j4TS@Nw{D>gMS6wplotz_(Bnn`kaQc6i8 zG2}3s0i(Np)JHG;R6APkz|%f=X<4 zx|mOODVnip3QFx83y|;$sf07da`m-Pd{KJR@dD-PA3cMVSOh$qR7jQ^**e~xzP7G^ zHnibJexwH))xaL71Mb5b!L-1&ML-6KXi}yd(vure#Evfbu&tjNpYW`akOFf5e1Uat z0>ub%Z0w*=@iAWClGP2b>jMFsE{wx75EibUY+bp7$0g@mG=d2#;G&(oauvjuT&i-) z>jy3HiYzr#1nA$Ls?t&(PBpB=-PX zI((lm_KrN=obE0t-(!2GV)gUsGIhItSSSlair`a~%Qk-(3lIqdeU0{H%BhMNiuy?c z?6f$!In32SaL4g4js@kF(F9}qjr($i%4)hnFG9)VfratFfyBopn@jML4GgpGosnD{>Nb+9pUbvcAoB^t7FABwcSEQ&hC>aKhIh|%E1eAe zWhQYj({4&nz@gZkajIhe2sa9^{abkI{PGtYV3;}^3D#1WM9Pg+J49HzFNC0;(QFq3znCATg8MJ&S`!l!A zB?!DhB+-YAUC{91WcI1%1N~Kvz2`n^X1yRqoW&N_-6`B$ATVk3390@8$v-2{a4wa-N6OSw_xVJXCyrZ^C1N8z(-O>?fU#;wIOw4K!mLg+ zo@)+GC&$yhIvrhXAPz|i;_zM<{wErE1uu3~IA}^bR!piq=CxvmGeMZyiung?^ zo<*sIpOEzeH9IIwTs@^^p{2JZRA=K-;=8 zm@s{bLt&f>l(5`hHciQu=MKt`J=13`Xa3*I5)@q*oy(jD#b&7}rD;jdfdI1L7D4t?2}J9m9pOfH@qqWkd*KM}pJ<&7tmb@OgRO63M#-O!m1Cz+ zx(1&2Xw#H-)6_ZGqnI(6^iAa9#0@)vacT?N#edw@S(A|NI0imBIpkrZ{@XdRuX5z9 z8~mP{{V*D!oyckS#!b4~%By9nVF?xg0^1RDMa z=|-x|0Av0ajod|w;{iw!^a>f)VC&IaV-&p(RuJfCwgeK&Fu=X3R6lT)P-j7+zoUel zGxjv5n}lQZF`I&8z#}A#QZLv-WQdqlqpdMdX&gXA^+NY(QGk7?-OKs*_2={Xyt43C zFD3fc2t1p@w@QgnvnBwoM1UrBU?OF}Xna&OPZvf=)SApA<0aM z_76&l(>X2~JPFfM<)`Vs1Z?#4wW{F|X9}mbY^-$5_SllmcO1Z2qH!R=a*Zb-F>vBa zt&yc1{e2$(#f%VJv))60Wf{cxA-IQ3gi)S}wAv`2l2mX&+1JK%&L?2*^DZzBtPJa< zBT&-vW{lmiIK^4K0hrmfpCj>8=FO*E8kr1Q4Qbk0W69aIRCz)W(Pf*V$w--PiI85A3}8Ec&^b} zF1qshS-^!}V^i#iXns53E=(J+zMq<-(E^^hPz>3jyf&SwajRDsdH4nz1-ji~C~*5R zBBL{tS8!hg50YN6oNqdhyDh8g;Y>JU;oy*}3wr+vl=$~<`Uv5L^glXc>dr2i{>Gk> ziEg19o&yICTXfxyJ(d_VQjC#yjx!si&YQjfgs4|~b&?8=@akwXS?rDsfg*?z%$PLF z7tiEFk?yDy3-_&XxZ-Y{f=(#)4p1244~tuW1uwBZQ__&#}h<41(bV+@$Z-845xB6dYrE)6A@W!6vC!K}qYL|3QBEXlj8B&*- zOA3!RhSQ8cl{TEp)sx{$Cm<#=Q63WopM@skE-nX|^Dv?A@1GdO2lBI{&73_>auxvj z=pq8lzRv_Mp7cRs2!W|ZtP1y{CvZaCAFd?jQ0aJklgyarxAY-%40yZ<D8xrSqx!$D^d09x%re6VYHK4l^y28_z<`rCQZDl>TndyYBvZQ!^ z;uEP)yD{RG(MOGkg=w?KkdZxJjv_w^a<36DMAK_USDaF4i{c-mT@Y?giDLzIe`M6< zw+Yx#J&*sJ@&);$y*v*31ld^-h|G*y_oj(T;lV0U)`>$Z#fY8#)5qZ07K6y*!P zM;rHs(YfY}K(4G@7Xk}jU!52kFJVXzUtij(nyinQSW0K{Ry=?e38&Q=TGdoXEZ`K?Y1SA{R1v&~dR_j4T-f1Xv16XrDu{Ri z?>3d_wm%KS`(upeK1xu<32t!iv;|GcuNFwwN?E)&Q_h^v zV zF_KXz2ZKuI$8)Bj;sE!}*kmd8p?~&dM@g#oe<=TvxY864usFle8uhUZRsPu4(!<>z z{0^_y^ueOQ{*$py;SYRC*~xB^6LVy;Mx^Gn9uP<`cG;2|zC-$1(PVT0fO2{=67$#< z)C@_KH|!U$5F4zZ2ud+3P8d!ith4 zqNXIvLa3{6AG*;G+P-HE6G)k^cX@PSyW>8Z@0=7y?-$%8<|sYUBprA?$8(uXR)C0d z_s)axI8vyCt_#EhM#dF-koF{%G6S`&jkZDTaxJ#y2wN$ON5IFe*mpt2s!m%r+K5j^ zE!Z8%=K1eKibgi!SiIn>!Q9v}L!bY>PEqX2?Ok;nsYN5BXoz|IJgkbMlhST~L_GUi zcER1qph{>gV50-Ai-G*?uzsB{2;73^!ws+A;x2L$FKE?1jf4090bW3%zlnQ=|6(bPzya*SMt%SA!A6Fq@FiQOqLI5^Ct0*pH z=uU=>=pS%2vE}yhv|!1GV)miKRQf?P62RX}E))bzi@~FFW&!|T(*y-bq`~l|7r%y6 zaOcmzIRCJzL$K%=^oKj1NmcxiLlhv+w#L$9IQ+}wlNU$lB({-1>nVOkK^nlHUt9vC z!+J7CAkyw)T zqUjZVJNF|R)i|pE5G$}FQde-I;JPcQsc&>Qs zLK|9DO-T+A=LXx3WZo1Af&ivhw7xlsQ_PWEv zHQXms4-S)Yc&tLRWD(7}IZpv+YM(xR#!H{K&!#YBqI|@EtAJqm-w(d$F%Een@O@UN z=TD5}@a2-?EWY6#N~RxlI(#necVX`ig4*L0)BWVC-w#ALWo?5#4y2M}>!kG!@fvLd z1IPxUKKnjLQ#Nw(dciS{$Uf>7={Q}njnlGO&hvYEtu+6fpCWByQ!nSEbH8*6g>=JR2B1ks41X3)ZH7p==GbVfF*LWho zVC~q@3;502B(>DI0RqxOAH>#AKC7xea!kkm@ai5Oj_{lF`6*7s)moxP@|vE?I-t=o zC}8kPzM9+sB5+;}cSk}4h6uE6tC-^sTya6N5E2|$6Si9YhyykF#|gX~eA4%aLJ2}+ zGzciK*%Er}DcBlx#;u|hj10*zCw^ciz)%xR0792I-4$CJ1aZ{^-FCX*c-WiAL3oK$ zXxW$|<~X6vOHztu01Hw_%XwZKYs?ryjE&h%O{qaIOd~x<*ukl5ta=qnUp70%fEH&z z15#jp2u%uoS#WLk$6p`4Jd2NM=$~8c^wZxQ9d=kF^$##e{q_gm@eTcpt*t_^!>t9C zzHHeY{2z}y8&b^U0Y(-4V-4%!%Y%!HokkTJ{jVEIvCBd1#t_$g_!<#&2O0oK*J6Kc zr-4{%Xbdb>V%-LsVc|%(01fA{ZSB(p7&AJsyPh*K^49hw&{O=~)WdYs*6Qu5`gPQ6 z+g9cC;WixZK1NPkLqp>3kcqcNCU$^qd^u3j^#>i9f*BnM`kti<^(EWD28W(4BeDya zO6x25nQ{6e!-pa6R2RVPc4M2c**%+Ota1O@m`1*ev zS%)ng^Ca*Xou2<8V6fZEKR3W^=-BW(6T!n|Et*t)efU zz~1oZTOK~ly73q^{N_7WqH4n%>{*A_q+xa7SwilpDV$YXk?@6$7z>AL$`rU|L^)%y zg<=xFPG-X;$!KVv;lG?J`o$V zQm(R$GJt8E{Rk!I0(}IZVE7w+d?JBxwr3dB1~~XAK|O#oOfA^3x~Wa&Qq!Ch=I9q- zEZ{YN>DjmK0h$t-sYu36W$3&Gfb=lK(AA92U~Gj90h|wpOm(P6sBkFUKA98y4}?6M z>Kr<~kZGiod3)_+o!aw#3c+H#8w7g>Fal(d+XV{sI|NX#7gSek5fE_Lt|XFXKf$j9 zhdgYIB&hX$-CW2dfsa9B41jDK>wzO<2#R(Qn$W@rtU@_0VZt-xLyd4CB0jycb+J-R zqxG(UCAC>?U$27wBNwa$+2V&o);5ZAM<8+2WBJlB>1ccluorkRdYe24h~W6MTX zddSj0Tjgq6JuBz1q3_u;$ELrH^U9atvg zJmVxm3}pZGs}EQ=9ules$eR@P`i0jJeG%gGiRKJ9FpruR4#VRP ziPzL~NeopsoGvtOOR0+w?h<{Bkz^(&r83FPxCqRVa~i2O+XbFn>#tC|qeIPA^=`f7 z=&**4BHtThtAGT*r39c}c|&9bR}DrV!c^O1#RAd*qdve?m00VZnkBEL{;ij|vamgP ze^a8nOsv-!+~G&8iHq``mWeO}$26Txp20-pY{a9gt1>zcty5*Q&_VB=xwWv`qM?{0k4 zz>Q&5!vnAkAIuO4#HZd&qo`yENcy;H?8voYUh5CPkn$@iq(8z! z%1Nl0{`3p#!h&M@!!M+iiNN$%0IP$puujLT0Dv>8SjqiqgvTHO>AzHl^qc-_#KVwST4=ti3tm@5@r2jU^)wsVsFZsWbr1`bVXm&#O93hCnOU8ymiX`Tg ziIr-IsscuC0=W7b)yzZ>xafK=N z2h_#G9I~Y?F^-qF!s0{rc!Z*R_O}~XJuw@;g=Q2IVpJA&cN(ImWl2AH&*Zfd(vfYG zN2_)3RBW)3qLGvdE~s$S#nY?FxkpGJGEgIe&yg_LL?;8_g)-H(`a{#DJ(i6~GxEaX zTu9IyZ3e6ST>tdbeYGW`7(bAl`#V2yM^Tx3~9niH{Z2+HK0=)i0(LV1P%=#5utG1knEe zv(5^VA+sH-wv9}*&a$n=zEra}#-f=+zwqNKsAfAIIr3htA=S{90T_uy0-v^CXzkSm z!2Bh6s}-Sy{@9_{5}AQd-GR)FH|3@TyW*2)QH`i!*Do@_TEiU&6WqK7a#JsIh~9=L z6XF5E({?2UdSHFmWgQ~7Dd~@tjeM`sq*8;C&T&raB3V6%Et8^yJX#?l(+EWd$Mvla z>|9TN<1jvH@)_G&6c&p+6MN0rlW9T8ZxovDa1=@OM2~>0Wsi(nHg~@(m~on6JkW9N zCke%u#+g(QMq)!D^PuX;>$1z{*ABAF3>Yp0TU#G>O3>{sbsGQ3 zdk%HzbKvgx;1~+G2Z zzNFT+0g*&uZeNFipwupoX}x%i;!>M`ua98~NHdt|HT11fDkAV1UWt>?am*PI>1W~+oa{(^V3YG$Q0V^y=s09K_K z!1Yi4F{ZR{LFir;;qu>UU8d1^4IB#e`YYF7}#4nL?vt;A=lIbKPo3_ocL3bsb3wq`{&$iZ_8MPq56 zh6?GZM!zW3wZ}@Oupba$JE$X&q+gUNQshC9~F45r56 zsBN$zPfhoL$3#?<; zuYD!g1XBm~k^#J8i9R%L-Bs*051MboCmK1}WX@Z&3DhQ-9zeIH zgwTm16I0BPhv|CF$)Q`6Mveh*Dtpe)qlz(4K9-Wn{|fk}bJ|%eY&gcH_!-)?kNvIQ zLLPNVHW`jgyj91Bm52|v&aDe*8>gmAwXGRQyQCo6x&=CBv=y`mTm$Tsf>~@z0sqK< zD{L#Z2Y-6k@WM8-}@Ir;4eH&}yY8 zyA|=rE|YK2q*+z5b*rcrbzlR=_8isu`RRGD*Fn^}?{?C4EWn1qV&iUmMQzNRUMbi< z$z)22n#4E6i;+OddbIELH-or=3pn(tEoxM+0@I^9Bv@LrogBhxcOjAp!KozijYtv3 zJ#7t>SY|;N99)4L9b@k3cK;O~xxZE!LG#C`R2w&WFrP=3PMjRLpp0+MQQ?+p@SXKN0*omFM=BcwBQ;{k%##o+0PDhzixO1GOf z+7ekEG0hz{*X@pq$gVAM4@Tbgws~`eDTkRDM#6z!jHFR&WKL7{v4ovmB>EJHaBO3GFW!VSWs zy~=HRq+l^=DE!}VkE<&j{ul^+C;1OLI4|K2eOyy~vV-@%ed;lCEohp(vTw^!2~ z5@LDNp{8`D3{`I=AIn@(3RixDt7Ghf zb5(+~q%)yCGl~ zbV8)A3K;XkRm< z_9){o7Kl%lv>JaO?aJ|eI2Hp_PpyACs<)&7#(ZyQdI*^#!iPbQHdEHDDDmX;(=Sdk zPOjmnu7H4dI-c8KuVznv(ic9uGv)2J|8{vk;*53Q&_WGgOUEn}@8qE%j9oHX7^b_) z+zrVjcFkw8gNgg=Fl2*Kw%w5JWD|0L{X5yOzjDK8w6tS#IrMwC*Kt)=yh`;*c3nzX zGb`^izEaJkJuOXoQa(bDNnKzwyT^!6JgV&jV=Fv`Ci(l`!0vh5*S}yLF6jpeFfvF; z{^B(9ZCH`O{{ zHcn8yg;`$VGT|($J}ha($TBL3x##+DPSVc))g>5n&|qg$eeN*vGNiPcHF6tpXB_*( zmK)cl#g9U@Je6M&-7@el$~`|iJUagTNW(6bEw{v6n$y|np`$X%piGO@+`BRK8;_C*`h`ux!W&p?*e*YfZ@W>< z<&h5tU4y3bE?4&WwdL3&?D`!rv_KF@ZOToniFa?|uS{z?dsrrpr~W4L7xPk5fU?-Z z!p1QR*xx<7e_p)*Zn2ucj%t(CDiqFOs)?P88X1wf*275rRLYUX<4*Vy)EoNRk@;`~ zA+Q!bGIxT!$Z& z>9}^o1!$=Rz-}QU1rRiBi-3c1k;yM?zopHv=g>v1?o=JMAv4?xt})S8=0RM=8RQ4n zxB|K19lhP|7i?<*zVG5vY^@g3+`yC?Y2RJ1+||0APhHCUgh?-2Njkr7o;BqA>J-v4 zLp(mx^Kc1vr1ij5t_5EED~>{ z0r%Q+da6UtvSm2hvC=_Dz1>@!*47U8ZT35VdxM6AthX(0!)C!}x1Sl5yWCzM?rT8i zM_0eh1iFO8o$7UF8Du}e=fx^!WW}$aG!LuC{ne0KDOy;F#*`|cp!eN^-tpMV0{|H{`ZIa9c zOaQ)?pgR&W2n$)ybH`G=qtb$Z?~40YYfsjZ_d4JNJJdv7*;ebAlU^uVZNG@MGh>n0 z4lTzn|8eX??(ic|Ci`tjjbKIW2R(^I(yk=tC?$5i8%!n$+qjQ6WHQHjanhK>Y?`Zw zl@|!l3N+Eqw%dCebkhOvUl7rLemJcDf2CP#d~B^5=L7TkdWIuUd7&p{c{CD9KQzXl z$&lm=o0|=xuv`H!4n6G>`CeDv@SBH29COws$&VXON==#lAO8 z4idb%k}a4_=hzinVFn+Vncp=hXi`Dc1=B|XZ#oS^qoHy}t zdAmn!&lrSTREs%bP&WK-cRfL_#e=(gGuf-?@av1~ijrkaP?s$ES`0{odo|}P&?yhJmt2xZI!N8&&%zt* z*V|;@zdShj!!o?cBc>`3;tPVk-ZK@HdNFgv9Acb$E{84SG!I45450k&g^X+UDA$WtsN1xjdoL4pnwo6#iLJQZK7U zWAOM*${iZil%MMU>4{F*Td~br;ZQr!^}tzBVFri*E2CEr?3iN zeR^bSEba@0%uez)BMetgog(v>L!t2&dLF8Xn@e$YywujP1)YKn^X4 z^ujeDHY?0kX)Y<198ZFpMl7TrSmOkxum(EMO7}>!362y%zCb!hzfR4WocQlGht7>$ za3%o4sp)uaN^+v|Wy}@dfGWUjl5|!VP4r=KXlwlE_qSs&ZE9|GZipdCuTyK*8O!4o zYNUF_K2NS*qts{BC!$dgQj76aHa@<*qdt z80iKcd`fv9hV5|K&^v24{BgpJS6|jE?a)jz8klQKBR+Q}W z4`)YRnM$^5=r|s7cdZ-J=*3~%|GN0N*WmX5H7Lsm;6FR8FwJ>ut{MC!Cw!9ALB9N2 zyr7gVk3i9HHbs#YuJ|%XaD31`$9>}SvbZy+I^V;0hW?9R{35B6eR`5gsNOL7rdXYz zrd3zut*uqVY*e>WRn1CkxaclDr*}S}@Gar$UQ1$;EK!?r1K}i3!MgCPq<95^&1ff=RZns)<9`(0kIO**}%BG|G}TB$C*L0 zyrmQ{^o+lo8t$`!gEYw3K-Q_u73+h~`a@Gq^SST4U6pc%#so(kBO!%}r6jUp(g`l@ zKhkv1yL8Z?UND`6;&A4FoMU z4RjLOzs`CPY5$Rx0NObGEt~*vqZf}0O82^zRPCQD)OH?0R}4sd2l{@P(k|uz*8uR@ z?_icLVa)d4wxjtF)pXkDizo-n){|#sqNGw8B2E9R?fV(x>p?t*r0{& z;%fV-?%I#wvfYJxsc_s~nEjD_#q=x5cQ#6p;q+~670l>w_P$P`b1-}qsk#kplj0} z)mZfr-LNp~yosF;aPK|hLV5{e!jBKX58e!!BTTl*s`{GlSl}!HkkN*(?8|9%03-MX za_8!xO_>ESgkCOyR!dJYR680{4TKuUE?1^J*j+$1N z(9~c!Cc4!q7Xw90I^djICQ^oOkFLQ-@uZr~43V>!Z&BiQiqQDJfNI5ePU73yx{+J$ zA8^;C&o4z~Qg*P<(bE2oMr#d&7r(sE{3|K{0ABqQ@V}q_{tqqpQ__Zz83riCNi9g5 z#=B*n>1ELJWXpYof_~6KP@uIprl>P9kIXfHu<))g9Y+@=j*YJDqTwf@Pu`8@B!El=NgkH<;GANR z=Uv%2bJ^hSN_~W;9)M^Zx+inb*ePn1NShUL0H1w<%oJUyPT2(O^1ZQ9Yvux8X-XO& zcoXH0S8x*SN8iJ#t6IMX3(x%#1FwSHf6Z(faN>;-w~X1EcQW+wrU2AH*ff{BlDs4LA9|ytlh}PJ zta}r+mVdy-ur5J7pc$(1pc`oIpzPQiTmB)vY2AUk+KVUDL!cEF?aI-V$61vv#vwH+ z;twH99SuERrrq{8`5mXxtD_Uy5!%x>^qZ6~A9{Grs{eKna+nS-PjEWkMRwe1jNeXloW~A(?*EM8g}f|jzj`C2QhS_3X~gSdJ7u+*qxJ2; zH9kCmUSp{3D{HU?q+@_>CnIBz?IOY8{{L;FX2*Di#;~B7ZY0gjg=of97up~P|XL(1u+dL8c-={DE2?v5X1PE0WRPJ&s{kQ}uaTB+-NRwbY zg$nWR%(XjvIdRGd5&{8@?cLdHc6N5ozMFl#^8I|6*HF?As@U$F*&nv?CsrW#WQsSB zi!YzdzIyLdkj{Y)JzSEgSzVNP3nukUVO;yJO@zB*o2%Q}6L^>?DN;oxkz4FcQ$wk! zdw8aB3J(>ApahOAe5~tqgHIF}mJqvi!9J=Uy+2ZG z=ok0SC{=a&c)kyZ54xMv=Y;I%U~?w>#pQ=nnB${c1T;J|Nj?`^d-LvIg;aK75ZBYs#RhT${GZGc0t!gd*cJe ze@BLSTwAGf*f9^7wws7HJEghgButWUM_^!RS?ZpZCx~$;s_XD-7YP;Wj_Q;vR0%%V zy17J%gpshG{f@~n$Sh#4O)(czIGpi5vK-^Up2z@?y@>C-G#}yzPY&{{E~gz~?7)8X zB+TexYFR=}4M2jd{KvJ>$T2J&Qi|v*UfVkgVwNa6+caIZDn(3lL&%?(pCm1!>)A z`d~9-T?(3tb=gwA25Jb)CkDwaFkD)oTDKriRZ)(KS}lcvze8ap?R=`BN=X#Pf~j-I zZDYJMWa_0 z6t6eUI3n>hai`f=Ted={M-dlWzO~3cFr!*msU%Ms$XS2IKq#E=12!8v*LU@L(GdW$ zJfj|~5d<{k!^lZzxTbNF6*3(ST&q-g^)i)!{yegb%5rLk)tIF){c4MB%Z-P5FNREpLf+`*PL+^&{)0{ zevx)O==@e`8xtQ{9eLr+D$ zN?4GxCH$qTwjtbv;a-}0%$%*Vhh{(}mFQ{>6tL){O`_gZAY=k_PhSkH`KYp-9E-C$ z%p%IRqjPoEkEI;R;htu-5|jiUh@*+y8O^AV26G)bUL!uPc5Qvnfe((Gy&~x7;TmqQ zxirqR-)cW$i}1FeJrn{G%9v>?pORS)HBqDhstwCN0KI!nZg|J;OZ12sumUk`sngFu zI_Y30uDr$HH(!*$RIOS%$|#|-dTNr|g6n!ZWl{r&w!8_>ubT?QZ@#CZ*HD`HyNzi`7R3OJPs;0$ z*0~c0}+7jA70PKr-Hl6=5lfV>jSY)gSSt~R^Q#%0-{ZJW$pG%MKr zsprDUQYxvaF_2IiApt40DW3rl+HC{DF5}i@oTnM;9bb|sYqES@ui%-cS9}NhB~~L4 zEd2DsN1M$EaO1&|Hhl2_FDLlR`TJhUaV;VeepNPHq9BwoJwjwIyTvW^tGp=@JE&JE zr)w%$_M3GH?djU{zT0pmGR(m8R(@WZtwFlY5kGthAM>AH7{#-s>rZSh;eY~dYn%*M zHm_RHNPI|M7cx8YLYB!O6)Y1r!lyk!qpMxzPudU@jG1q}b&vmbw+ z_~&w6%{FUb@oVF?VclnDZMOsp68l_uCE#);^lDiG?OsDrnVI>{bc?AVcLje22e@c4 zfr&<)A~JTfn`X1yrkKdjX(j>>FXq6R6f*=uca8WYp_op>i>h)Bp0wI%g!D7Wsu9cN zymwtcYL}K?Rv>S@h8~{Ib;S>o*O6kgd-B3UZ8*qVSrI@-CaV z4{fo9lfC|((hW1a0O*SkY#87-9jG(`-J?MhHrZk8BTdnsRR{suO?nD5u9~On_pT|8 z*A9Z)mM#8eSb3-|(&mNFx~iP@+-bNbTX3!ihqp6{v!9zz&5Lo+Uzc`-_a^R~Ah+yF zJ|wUfF?4QIgxRu)wlu;e_5(is0ySfN3rW=}0&b9fvtKkJDwXh$>Wga*(U=Q`@NNiF zBdY_4!fewP%j$j&$77+MPUHi*MR9-&19=&YE7vZXdP`BYN;pL;I>;NMOz>1>-{HKV z9)&P*Rj(_?_^$T#;j)`rlkhdVDu$H#sZR*N{BKQ2bSM$3D*r)_r#m2R66V2hGF~$Z zWr-&X6%nho>K@37n)Ui99K}f?oQ54}Z%XBw?snm{v{}xRJ&|SFd;Ug8_Bgw%Z*Gar zg^`F;!Rmppv9gN-Q)4Q-Ygu|h=O~zp-!Zpbqqv!Zwy-D_*Pl`$1xpc)0@Zuw)LNA& z&16O_N%>|>LInjUM2poD@HqfKgeE9NJy?z-hgtmEF1ji$e!hlx>BR$JP=0w7d z^fDvLwA%estLdQOLEP?G9H9EL;9@#NhYjB!RyMc2Vd3FP&?pH`9HEh2_^oWjDO397 zdp6Ss3J=5gyJq#$`!8Q!d<^!<=#%+;$6R)^+ja`)LT(fuIwR?BC&@doN8)i_0Et$> zwNLo6NSCv#N)d~=op$5zW*bQbCup;~xh+m&Q-ktR?3>$+I?9JG@^g`nrCEP*Guq3J zV-d@TJ50E8;1l=Q^d>W6yo2XS#CHeJ9<&rY9=~##bNg!`f8O_D9Qhucc`+jr&N#96 zD_v=5#>ajBAAlUsW|(3BJ&1^sU4H%H`xhTwyf0BI0z<|qPz7A`{UUO%wOcq3Y6(vY z&x?w@huJC^3p6h_IW^%84NP=)W6?peH)vk?2ReHse5JL^#QhjR`g_LbshAZ~! za(EpXJAf@W!>j5s5dkC{xh;PF!3R@_5ObvUc|e4zU&Kq(FK1C$=Fk9KSP~(U=Hw@~ zn6`AkknZngy@w$&C~5IH)A)bUjXyHs?-%eN=LYjHyN2)88TfNX+3krp6O`;#7CbGN z-J?9o%|4O8vdM_L1yhXg4%^#PF~1^%9$>3)N(M^Bc7^(kuKImQ^G@l30f&@yVHdxv zN|-fN3aHBa^9^8Z-CC5d+YK_A88DL+jtyf1LUj%(sNnc~5-WU-k9db=mK7Npiy@=h z1+=VDXh_0{JQrT|SH$f4nY$k3Hh^$|A7KnzR^^IKxC?)bD;&XDHpP*In%%mc8D$d` zAuc#BAQm1ow^ZQmqnhi;c_%v2JbjFc18D!*02&68kUJt$aFkGl#vRzk6fVsuuV-Y9 zdM4%$1xU>iok*|mqTxib88dmpzy4?ir5K8W<3G7HGcm6(zk2^O2owgEm0!5i+Rf-F zok^Q{40+sdArTiY;9{}NdxsH53Vi(~aNrg`q2AuVImur0-@NWsz$wbkyk>O{@fV7N zlkOsD&ReY!FLaM1G@jJ{(%5TL^wpnxjXqH39awDKwKq_4bS|bdY%bH-xZj&Zu{{MI95MFBC|P9clT*o0&2o$Zr^^@!cSIdm}LVHf%JV7W==$LIK>e7LHFUn189 zl%>(GK&PMGHrs9A^>G!;a$E-bkyEBnF<6H!;z9pFQV7(o?(q;(!EE>dGHdwtG#GJi z^85`8y_`(fvr;jsMSyGZ9P1Z zMAU{)dM?VXMA%dn_w?ETupSv1-W#7tJ5|kM=s*GmHvN|Jh0C`K3JFR{; zq@QU^Y2V0b;gb*8(k_cAfOvnSCJxY=$Ej>IQk?5VHdqVOpt`DEkc^Z|@lR|@s)f{B&@_3XNVtE*Rlkjll9xS*uo^(inZ zr{u9&&`tN4oakfS_CTqAuNFIO3zvq19ng;W%?Rr{YK4=N_JC+AjEq#mofSvQI{V+M zbYTW;V>T=(S}F9MF3Yrf*i>uG#R;M=vG&<1()E z&ErS|RsFm4!QY^f3r0-*3zTFvc|e(JilT8kc)fbXSDYAjoca{b*cPN9gZHf}Zlgnv z1#e!Kfw@nx)~sLThM98b0CLv zVf#8BsGc#`4f^YU;^D+skw+>LZQ|v-%{_(50WBRUD#~BbK1UZKbcADbh1J8;QAj>k zBbn^HgoL~m!iv4Us$>vt_0+;sABfW(57;mN2&uC}m-^+B{b#zOWyaCcx{cYQz(}+h z&C=1ia)eAJJUu5hpR06A==(TwhbQ4u3@1SWoYN@Q`tkH7qZQGAP`5v;{}zP9uC7y_ zgr)5-X0EZ83|S2?URCK`Qi3NnC(_^XIKj=(H_cs|Z5oQ9!fy+7a@gi$6Gvruws!z5 zt!r)~w89}H5kF4K=^n;mCMcjX3cwqV-7H;{yGly_5Ys0*RscfoLocc|E#PR^UBC3! zC^1jfDm9>w&!_J}k|h|;=a?gE@Ys@wk}SEbR=K6gKk|rw9zM*zn}3ijYWm@mZ>C?) zzJT8_oFTKJckuMJnIr3bv;JjGw=TutQQ_le09{Gwch&PvWsDQa2+;VtTYK886>1_D zH_XdYc(q;znga+Uz2&jPF^?#J)qqpsx_DX>`r&l?)pYQ5icCeCh{d8x?ue2dA?ts_ zu!syM@6D*2a-|rf7)EKx?`YyX&JT?Y@T`;QW{r!!a)Kci#Sz0H(aS$Q7*8nkBB;OT9=+AW8|3AaWk!a~amj!F!W7 z8C{f=eD!{h0Z1@ZE<#7AmNJJ=nx*tVY2)%G;L?JIPEYoV34Q6@$q2gLzZ@bXO0@t9Ti$_K%1OD#|z#d1$Cy@fjy_oNY(j zO2v&FJwWzLs-2WXk7FopQEs=mh3Sb4{EhB&Zv?cFIfMxnQfUo?BJ+&%C$t&Tgu2o+ zb!v#*6CvQ7syy)VDXWd$5D@6sO=%fS5K8vXxGom}B7||vm}tVJ3LI42FF8NnoaLaU zxYF|{17)sl&Dmy#J0$@m*#$gs`);$G{iDae>9O{^T56A5H`{m>&!Rutc6S8_q`-wM z|Dsb!r?%5XX@JViC)pgiX!AVA7d8V0%@4m)nHW42 z;QewUxg-xa5ykPsw?F>$U#~&k%vaJ^bi3mSVgB>$;Pc1M=T3=t=(Zw_xXU2XW zpiguXu=~ypkmC2^`|QQ=;;$=IPRy*_H zLWfO0n;p|vfeH0=0>cf#3mxHqc>UAa@6aWAK2mqKQL`mc-L8{0P z006i$000mG003}#G-@w!a&L5RV{dFOaCx;FYjYYm@caD=m41Pm-o$C9?a;b27+)Mx z3>0uuPm(#rAD`T}iE0%UWy&+a+-pdvU?x9-h6_Ol%8y`YD6|ILpMx z?E4i5JPijHJVU71Y<$lbQN}iL!fr(*5*~cN+b6p?6;>#+ZMi-|@uft3X)znRc1TJR9vWNCeM-GLF~;M4dkCvZqf@pIBCp?}1g)W9K{p zO#%@f*aZjoa6h#$Vm3(}GS}Vgvwb35m-*o?PBONSd@o)LH}ErbB#7dxG!FKea3zK% zDbW)@K2uaM9WVn=*6!-ZXCgud-O|`v;FuzX4(_Y9=1>7B;y|=wM-0>_&Xj=^a z1u?`AiIukN=BmQrArm}F_~B_6QSlp*y5Nk$pBf84b_cLd^n(H$vMh$c=&C=75PoDa zes?U(df9)~cdz*FIfea?&Tskv70_QLr7C5CX3ufcYobkX=CM~N!oAqz>Kn8TqLVs>So#EVk2QBNWKh>YSWsR># zMrzcurv1V1{YxWZ2hJJw=VsC#uy7VHCKq}n1TGWz%DFms$oFx}dNmmu_#K#O;mrDr ziLuiyNV)+o z;5WboxXib(SiOCg<^(QIPU7S&PFQim8j+@4LNi8C_IUos*o82g(XvBD-NEGQst>=X zau9Vtft3&y%#-7kSZgRMJ&3~)wyD$_^hcu{T}Pyc+@24e(M1ik0T8Yu2%GBxn7W4G zBf|W}_2OdkdhDtdQ^QIDi(;k*2>^m2IFbvs22T(U6Pe8FTtkGta7NP_WD5`(!!-~` z5Uj^nL+5o3_#T1Ip95#wsLT$*7sI+0)(8L_IsK~|Yz{E=K_53|mhM-ca)wgy5?0py z`n*PWYvwRNf|3?cM#F*Ah!hy4kuz>Y5e0L>92U)tJT;M-*aX7M{5UGkhMY)~xgP&M zp1f|BvCXl)jxX047~z%k5UWI~I)Dm##k~(K-9haLm3cY2 znKERj5}8{|T9ttlMtCxT=#2LWGKHBFQW}wJl(g41)$APT05rmF{RrYMP4dD|ivFP8 zl|pcSEU*cMiiKvJFvpSQT4NaJC@K#$;oQ+gC)bPVH7#k_8x~ZFe4;8^S2#V^4tTg) zbM~>z4m~!GBhitSwyM!8Xm>o=3q+NF*#Y!oNxQ?&u+U2OqNS*8y;GDRQMaX=wr$(0 zv~AnA?MmCWZQHhO+h%3w=^m%g`0wo)@emL3xc1z!=3X(s`U5UeNgKgTUM?t|WzSKH zhku3AMNrd)!tw^GkY&m|5pM~9$a3k@V5$4-Wap}zVd3p+c!tOw_XbhwLkh2v9heN&3 z`HMGC;x1=AW`^kY@G(RHp@eXOVC?>?m|Q>0fSxAW+mubcTI|XY3~(!s+zqrA*En>> z(6G^#G@#T|6d@&SjNT|ZcHoYJDV0luWIE3WG?N?@OC*7KMc`GABT$UFpx?3}O|yus z&k`q+^qG*G;aTR`pN({Zxp$rH8DsMxV7owE@3VfyK!de0Og7uKlFeY7RO0JRefhjK z){oq>dlUhood5ymxVLKdK}}Sf8s{lq#GqCv@hFAEXd^=@yYGDtz&}FxxkFs>AmCpx z9w`<^2mX*0q_Gt%&S?k!!^i&O;}-8hV`iwSC_JL}*Zbh!s^44**AMZMKY4E=NgXr+0 z&y?Xn=te1E!t%vqv2su-OIz20e`wU%yMV#AHUrZ8UGTC-^KO(l{V&T$8Xu|CuWO} z&*^caji59?+UP^=fwJi|DDL9Anz*nGEPu4RLuFB%GFnl0z&o1+=&WJ9`3}sB|I_qA~6-(&=Aq zplpM5jKE(gk+~c$ZL^!i88M^o|IH<~o)lv)H=)(4p{W;_fAMl?_%QIJ7Xc{eT1Llc z`tclvN)NPy#_S9F%qIMY4S3Xf`ZM}(IW;REZkZ2W+5{tPS>;FS0V59&LB2c8U&V=Q zG2^rA&AwPC`YqI~B_yLnzYW^CAJSsmiUXb%2aRFu@I*~5mDJ~B?qz}^Us`*`UCHb1 zo-1Odj)@@9*DPf)ylD_kU!73owJ2W;fAwjJRBW;48tv&U6p+p|lB0j;AD65}cVOYl z2k6W)koj>oB+4lXHh9IaXI)xtj@_r=tI&6n7_8o19ikRVjvdY>UM!!&(hUX`HoDAR zuSMP1HYv=dP0G_MuvFEW`!iH?Qt#0KuF}M$M46^jGg6} z=*X!|TrLUC?damvEo5jL!j3RI^l^emg4Topk}ioSm4#=DNE_Bjpx|Z10-Go~h-I2# zOUl6rXRZFBzzq1$P5kTml^&qHyrdBCHA?cUK|mIg@qVH#96Fw#zL#E`0n$r4MIe)) zM~=OT&ZWEiADR{LUetT)McW##`gF?j_kDC{0s&T3Wdrw@Dg+j>$^^A+N3J9ZB(h5b z=UpBTHS(`Eb*{<;G$?YDULYx)bql0l)90N?2&=?`m8EfZaThvTh&q}v9|^O6uzm8q{AfE~SwEfzPyZHp)pRUvpM9%v z^6Bcy-%%$~H+Seo3I8qBHj z&n;o@Ul6P8YrZel1{ za1^6SZy5QkCmn##L{QV0Cnee)zAK?*4@yR>m!|PWUB|tf;>U|VkZO(aK{zJSHS7WZ zs}(Lo8&FcpCM0uEz*_0NcisDoz|b*-<|v_1$w4bKUcY2}k+cvK5wj5>sP2=D8U^v$ zo9_H4@4F0V_JiKrfx^|E!%_4ypQAuvc@(5+pkkDn9}mh52$>kUTra|m<>;_*9mLvK zGoULYpljp@5iq8!Lg1e#wY-CS{RFVPaeK=17SayONIUk7eXgdfKsO$4@-qb|9W?4u zt!NpPOsC(&CQ=$0plWQH=;W?(bGc(~^#HGXu7X?h>5q-}AFwrVzMb(e`plh!!p_MZP7rMUk>OsHt&tDi>m=Wm7~1bf}6!1 zRRPqUBH{d>Z2yy2USjl5T&Nx2CKD6@0OFVE_S^ge*n6;8(mPq0{Zifj3+q&^Ha_!z z@P{;QBxN9QIP%4K{4|1w@#?Y=pls<)CE>W%p}9Rg%(@3flwq`?tl>Pn*1J>et?S!d zZW`5Cgz#D~``?Y7@aHnFmn+q36_{N5FJrXWO&?@f3{lL)ANjyvT;e;JE8*QIAKIqt zpGKZs_ErK?WH6s#AJT#ahZ!2d?tX$J0U$nd*RbYWh@c2yf6S^?d5{-iV&Xo(`gWJO1hQ!98)d~MkI-V|3V#HQLR+PV2tOkkU;PRYxJ6I%*X! zOfLU}?!Zm!#4=z2c%JX?XNy}2 zy2BPXG&xzp$1DjEz%;jq&!C(|heXq7Wj4_UPMlyeN*2YmU4y+-vhM|jZ6E_IOv{_9 zXS@&<1IZY8V{k4wQKgWeE{wvPFX2^x#&NRlY%~fW;Gyf5>(OUY3Cp$N5j@Ky;_xmp!0~LxGD9C`ZR?L$@4_;}W z=fh)12w*ymhg{cUXuaFij0hye)B9sKS1Go)4ewt%Q`fGzvc)#jlLBmD#H2c1oEYQG zMwmN~t167L&u~1}oFPP*R!k6ue9D8I_C&TZ{t+RVbcko?ffe?PB0fVI$1pk2Kd7wh zd0haapo5G~rd;;ZpAybe3NeJwRC5tEk=WvTQ&XTJlV1D~ug1623QMEGT8MKnk_l3z zUh_n%9XL*gvv)2LG9-qM*L#$fF%-x*!93xfGesyOM7`M_?NJqVgx*exzKD`9ru;4M z?j)9Suh(lXs25xBniX*Rz^78cSGK9@cKT0W0{HYh!by93lh6;##mM# zDi6HFy8~NFJ0KQk`1eON7TPL!UOAkqU&zAjVLDmC85SA`vXYK=0{O$GxPv0OoNl*l zWy`lCG}IJIO8!ot9|X)uP&s&*I5HWGAEOVEH8J6|;>di}t9O7j6ITEz=6Bfb^^ltG zBDzy$^?}V{bnmzcx)we`#sWOY_wd(YN>OhxCm>fJK7O>N;Fy)+m!A(_H2~7h&i8J= zKXeVq%R)u0vwUxd%MC2{#p=aDbZg(i^Vzsn-s9<2HDIIDHBuWkek^yb{Zo(@$t37H z@>D)%CZ&v8{}O&wn$KfGl&|8E@!07%PFkocAE|9e6IKK*YUcQ$cycB239 z7a;)t3+Pv4H}1O+0svt7tM>mF==XnKG&Hd_GPf~sv|{{K{$e#PyF~^R-`m={0Vyb* zjhM_f6pKE60?v|Vq&!9u+k=*!EcS+M8`OhdC-3oM63Xl`A^Oo%@6$)KTq83|_ka`+ zEXtX5VV3bWd5S?ng(=UQTPiAF%8a;!)~zw!*&(w`a6EjpQaK( z&rnRQy)#eIAy`B+^}QepB7mY-3tw+vmY=6*XEfh8Pdr}^-fwokcSPCPQ*yoCLP$>- zUc^Dg0ypk)0>}Qbbqq*CF{%2Pm2|8m0NcmgVPN=2&xYqXvqJ_O@*u;;pg z_5u9nvzJB|p$i z$|`df(mWUMiRprO6jj(N?qpyRt3z2TsIkJFV{)gGiQZPL2?&^8QPcp~axOc8k7HxP zn|Y9EFV!o`9y0Nh0KX=6%zA@{3V{3t3n4=ijs`8z;Vkha$6!M$Vj4r-lRNjC25pV| zDX4aLaCPJ(C8)23Fm>PEGR@hQy5-D?xrzD((Ucwr1-le8;XHb79YUkWDqkoffCVF%3N!+mMv%oO zG9>{jK+wSnn02eaA?DlxCkkxc^JD`g<(ROF^Zew`JLjn#A~m5Q3oP>t)SYKVmE9k< zObT=$OW!yk>53y1zfu37YHSkn%>nD8R0$x_09?x=ZdpLVYIaT&hD2Fs?33bHmle7o zFRelxTi7L6GXXRPv`wl#i`xPm#y@qtrq3%uj5p7hHh33;NA@S-g)BJz?GCkRKzUk=>lp3Bg2)0{q#bU|ypdlSK?xlbO^L0mOuNC(y}#jVC?HBUad$ zt(d1owb|rRMZBiprq8tg)pov8Oh8vzyM#S)5nbC#BaXP+dZCmY-w%1Auv}wTh|z^M;)M zws3#Kr};f#B;a|1e4XEl{<-`hnQuOu-fMyA+j zSVB)^c+%~FzGmk?^|0KTf@9eSBme33W#$ zPkG+{P=S)7a>E0xo6rnPba}S$ER3ziSoFYP#w1w5PO;VP)3_F1)8@}%p)GjrY=F;2 zK4RlbjLykaUxB2TDu-k^dqME@*?=mjx%@EVm=I3WTT$gk-hxZ@5RXuS7J*J)fzDvN zSJNUz1XE%7p3dZ3c6U%cTC2M`HDu#!hJD9bloO3#-0fxB910*euR+FA-Pg~`{gZHy z|8osV96lGnR07MLY~xox`#Z3w##J>#C@h*RMtUO}a|%*m)sb4D9=fL>Sbfat;$%X| zS?b|Zagq{S)W+X9iW+RrQ)kUCOV;(y1zkxE`wlrE5OnzIYCS?Esg8wWRBWnv2TW38d6GvAQ$Nw&vYaA=5P2s!Fzr*_sfv`1X zvIhE41{8#}{w?DyCx80fXCBs7sn4TiV;Et6vbVdrvWpUJIAq*m(^>0p>RUToXFHQ) z*8<(}>*Qq%o&SaTFnd1i5nG>6F{Sf6Wkl*dvS-v$XiU(w(n89;Ya8P(a>75q8Ka)g zGbQHBL+!0gsC5j-s8)_QY$VV#^&nSN?+K_sYYx&R)KeiSxwg;!qo)ExQodH}vB3K=6&5e74K9%UOed=R} z>TNzhCEPx_{7f`zxE3c4Df!QJj6XLF2BB{@uv%0bj2zMj4&A<7v@Tn47$bD~9#&lBaM;nyy(6X=p_r}n z-x|YmG$h|s1)`&i+=$0b9kzV#BP<7eizTDD1*_&FW;}bjr4Tuk&}pCj_(iW?_tq&>#OLqIra0Yr!Z4P>+jEhI^6|h z3m-2Lqj-Bu?pJrSe!)n81^fyyrA%&aK@ZQBU*FQWsVFoWl&rG77;Nbt7P~K&OV^2F zOk0DtH7TzoC7b*rnr^{cV9cFkbDGWs1gyrVq64CO=+G5DT(A1*G7+Z!I|N z&tGprX}n^ypR_Di2sgpDc+DH_=wA({k`3uWrSbZ8pRDt+_7*Y8i;F$Pfn2?3N?AHe zg}DP~1S2|i?CUDkvCm;b8DKJ`?*<(u{H^uL*IbwIhuI@dI7<;}5a$?S*USmaQm2dM zF%&14y&3frQAbt>2K`2sOuMSpQk7db!o()|$z6S|%F`&Y1~NHC z&Z08vmnIwMtGS*x59Nnaqk_G&v%l}P-I62opqPFGyPrgU!{w6z%R*W4u~P9?8{&6$ zNCZ?obFJdf7}gwjyf_UWq6RP?0ymJ!6-X=SyQPn0?tqVu{^Lej0nmuSdH4%8kLs7a zZV%~!X!8QoXH<|z(k5fYzYAqFXA_cAlK|Nl&rG}xnHwQ?^?JzH(JyTZIqM}X)*Dfp z=i}DCP-sf!#|U(=W=-n(Hraotvi-gER)bdQO|_~IR59_bJi)W!Q)goBpq$i5xdvopAP5MVn*KgX1*JEdIoOOd()UA= zY56B7EUp&inyXsZkD43opxQ565pqFGkjhcD78g!}Mem3m9I<}#f|Qn}8w|)?5a=JK z3Tw!M3t%3lH58&S!k^(pC>kaM01<{baWG<7VlRFqEJ9ing>;=zbvDgS8=iC|GY%h5 zAXkmuNI;(2MhbLLxH<7R|AHFqJd~i2bbwW!ZX@81QEg7{B(t*!V90G8>0x^6d5*b- zj5n<2p6GcVL$fw$F`?Q~wt-dbanaXMy45}oS%jR2RnBCdhyA(r`LG<1#zLs<_diQ;4Z7Og1Ryc7^?vRQ_MN6S3m8e8c;z}UminX{VT^9 zoWy|9X32zn!Yg8Wn*=ws^R8)Tfrcn115HNUEJ40xc;6W&+9Bf(}lF*%Km^ z9z0oqV#8ptIP}Nt9ol>@>_BK8AUeb(`MDSw*>}&d3#|eL!P6x3x<>R9HYinW0BNo$ zLb(J4Yf-CzFu45-*KeP&3d)^PuXs*|B3*!AT!oLqE)mlndMT9z=<`ym$l)Ej0sCj{ zOgAR9#j~vQMQ(FDaR+CX5f%fF+$Q=k^xO4Zp>Wk-p7&P8g2Fb*Mu@|&)ZCkz3g36m zF8y>1m&p!qE7$rtpCbG7@RK>PV5H6y!3o@;n**=cZ}6)fE^QMxf%rl5zDd&JxzZ~N zGzBOMbp@zyQPF_E_AlQCj7>2qJRFxy*LsXoaTmUwHU$DXPPPYeAbrV7o6WR|oHL!4 zV74K{n0f2$d3PM8JHgj=nX8h3rtp3aTp#B0^(@ZCJR1LJ67DN}9PDJ%C62fBL#=-Lt>c~%7yNapReYcFMbHH3B%4$`d@Z;_=mf7{NtQ7AUu^ySz zVMb$joKe4i@h0tY_lpW+4WI|yR+$U8TP#>lr}aMOmY>(WZM>#NcJVwrC!hQ zP^uLsuj+Xvqz9|8(tzH1qGm{Fh~UA1z=+%s*0d|CjMw#SItQ44^HHrGz<~j9(&hD` z{D8BA-g2mSqkP4Kho_mRnXSH(d^eu{{=H`sl`1kUddJD#xFU!l@tOLRCRf48vcMGC zbC5bu*jJEkCb$N}NRH9rvb6kVkpPuYW7A8eIy=5uTazJKSBKMsY zhXaj^C8y)+yXd{VYrH2A8BS7AyPbc#11k7G($%11TRIk&n!DY}fAy(X5BpHvY z3>N-6x4IJJlFvI!BE(&6k76aS5=mnRuo*bk#=_@{DIbWE*}gtEAUo((vt7L-y}N^5 z={u9>gZoGHvD~Q>gs#*CY%IW%Mo9;!t?=8En-loE$|$)ip()X}%8GOC@JN6rbV@k{ zXMzgB*Gw{Cdr_ypsJWMg0~CKi4;7Kbd%5J((a0HtaXB4FO;ZI&Y*`PSx{u(E4r6lZ zq^1>6^Np5xV8?^QrW8h8al^o9U)+JKVR1}6eB0NF4n-c2Ev&20nsasPZV9L9)Cda6 z%(vaHnDMyfg|-NV*U+uDEJ3LfUeTT74Cbo^Hu-r;G(f5wqzvi_d*KyRG7zs>tY%yL zSWQGOV1BP2tSA22#_h#VpRma^!R|FNas4De~-wWS+GYQNOdWL>!nnp0H-7P4m`fan& z7fXp|EheYj-SN_Ebd!Vr#k>!GKZbsT@q+S4=CD=tT`tdArEytvw06*OSa9WJcNERx zfyBB%W6(43^4`YA>}?~k7XcNDwhCNkI`u+RG5 zw3&_pPl>O)1i3fdpeiDSTOoTdf?H2@3@@`9%&%YAYApQjgtsZ(4iW^~)@iV)2;e6y zK{uxAsIF*Y2R?b3bJ*Ls+)15#1NhA}HfL|OsrTh!kS@(2NBmN%`-Hl=5K_7(AgY2I z)FRUPU|M!m1Qh_%T0WlHi4@N*LrS#7*u~qZgR;p)$j(EcbVa@g`(>8FgO4X8Qc^_8 zn?tobMwomk!^#Rfh!h{lfY(Z~PX-W`MbA`#Zpm#4I~D_w$(GrLX!C}FW(hC?^!e)$ zNRnbCL#aa5Mcwk++dqL-M~dZywY}5WLc?v*Tte>KCi^5zztZCH z9dzW4XB#+5Ah!mbbB4rAzVy(hU*=Jz{qkYIT>$gOc9@Py;k}nh!EI4@6_Y(i>edNY z_;kMeS-Sl{cihr#Z}JAN7P1}ieI&iBmTS?xBnp5SZhx>D)(PMs4`Qhjd*~s4{K6D`BEwuRUOx7ED*?)TdtAC-r03bQ*uQA)GT;0A6 z^aC`<<|ER!$GCWZ7QAv8rb$+r!^>=-birgOl3puSiwag7%FfAs%(M~&5^%nCbxXBb zNe~ces(u`H-8_?zsz04h<^I#6NfUelKmi5-Q2VuMaQ@SxF}3(T-nQ`kAA4q5ecLvZ z0j1}ZlExEQGYF`Hr5tIY!n!+cNoJ#?(xL%}y{TwKHdb?wAIp#kwB0nKD>q*MW|sC6nEEMO8Qpk1sRtNtz0l+dnUq=3Lh(!xu$W|e zMbSbrj3t3lKF3l%>XdmPrTr)vOw&SLwb}z)BRhH}?sDQ3Y^^(>$RVs%l~kQCr?!g( zHLCP4|DT6UE@|$6Ge~Wg!psYu(d-N&T8&!`RT$l)>X0dybre($0nD@tY|-1$k$cUm z<#TYnDRMlx@VDcgac^g22Q;(eCgjdXj-c^1K4}z5CFo26+@*fR;rs(?BS`&-3-WWJfktsV==o-n#za zJ?Dy+N4N$w4FD4C$KkciX_~|lrX8-=uY1B0wU`y7yX`fm|E*nckVL;h7zZT_7B9Dp z^8-Q@xFZs;JOG2tji*Nt$YFjKPeG_B0w+R7S_TZaBs$4CZB}hXjG;Xp1D`=A_@o^> zbhJkpj(twTnnK&|?RGgbV?zMnzcSp2A=ZA~eR2ryjooY3ctdV9AKf0ochcU7?we~H z{0awVZZCPyIlsAhB|Ww4K6W?k>NxxqeEW!bUBJXCBe#zN`^+lre;Z17PwQqF8^^IK zmC8^$J-P5x_Goh8AQ9*h(ahRd0#4YU_vFm$Tk2}s!w!4zB^EYIGJR_iDAys}O~}^} z8TtYG&(%a@UDJ2*yP5`20RXW6b2VAm{+i=1HovB)iSe&Deyk?t^j{+*pK9iNhRh@6 zvf1N+l74rR;1hs!Si?I-w%>w^SP!@HG~*4%Y_LP1rgzdx9?c&w-77T~It5L1P~slC zYB0wpN=h+IZ#}{My;pu)^6KCx-RpdvvRImrnpwg@`L4L)G|j1K9LiaM-Jq5Tg8<>D zUEA+lH(9GH#X}=-Jfe2vvlo6;Bb|2=os76sw zg86Q>=uNhM4?Z-2P2vKd^}S`619%NkZh|z`wK(7-8bqn=%TZ~99mO=gpfG&m>nM56 ztOpO4$RM%v=wK6{VXNk(zO@u_SCqt75WG`-G<-h8b}5r?x%1NX!3y^OEdG5t{%a8P zXnuS9`aZ}Mf)0RL2^%0Y!=aD`ty1QyQd^zZuxeE|7%68b_SN!~TDgO?z$$BycPj7h z5e*S2l_L75%+KPw)s0f5Z31$AhPjZj1Bfrtycoa`f|i_TD~0%HtZG!S6GgPitSA}d zRAq({4in`Pqgq0DN;f{hze$vr}r4 zJjgcf?e)l%Tn$*oN4cd(-11Vg8S`2wb^}>|`Yt^R*LK$))}eW2o5;f0P2MZ#^x)q)u;z4E zwGiIDdMMrf^W)&@E%snHnR}mOrH)`r*T^MK1kDS!DaNpv0KbST=pNEU#*KpnRRf?# z_{xKKH5^+tz#(xz(OV{G?OQsrISao;;~x!+TrYj2X9=kje0mYLPIWq7-Gk%E*2hPw zF86bAQ-YNje*+J8J0e!{KfvHqO@WY|0vkSCU&;+!zUXJHF_MMuT@3|@0+5G65B7UL zlLlYBImV+T%M7fb&j2Dm68Pv zc-BSmafeC%xGI(9Uiaev_^Tba^RfHCbe{D^>_@#zwcy5Y{wxepR&$zcC7u36YCMwv zr$XR%`2X&l003nW006lEsSuV1t_FGrj*bQ%|9u{ktSV!d$&b+eNUfU1FE}>h02fd5 zM`ocSc|&Sb$+(*AWo<+gt)p}3x%bxh&VV5;1)KS{JDm*%y64tQ4oF}>>XDbf9IoA+0AHXCm}zR}2=ir(FgeHIMmPsi(yf>ZR8exFYAl$DRF3T(jZz8~ zN}q}7j?77wst9i1168pY)00+;GKtf|B=T{ak#~S=uKVT&AG*k30xl;* zUIvQ0fWEv5G2$-)HHIPulxCQBpkheQsSSp>1YWS;queyig|DI0D+tmcICaA?ZhV<9 zs=sgxbN>ZXtAl{*p-I56D{)$H945jLnQEX`UMbxMDP`SYy#& zXHNrPykYq-I%Gq0qbFOkSKTD8s$*j;nSGH+u07(ltmp827?3`~e?HU7wAjFpFG>7; zNA-d@k?sN74fc8a{4`Xs`-{#WG%}3g$e$Wz2Vh4B02vlX>?3k`Ahu#hFYy_6-UL)i z_w-V=q1cj-7vRmML?CnOyRC-s+X(t~kk+1C0edz0;%m^5rzWxOoux~wF-xr4SEogX z$ZZCuCSeEFa<}&%6Yg%KI{9XTtAuF%3qTm=ozKp(DNFwp_bBWl+OWQ(j2z~8-ZKKPY7t*Z9`$$Oc&`QY&_|*f?+mbWY(zCrz!MPGO zi38KoO}t5a759sH{QeV+uOSBvz9v^9W~-IT5I9&R>jk*P{B(pe2agWoIx3|lZck#df z&jyDmeET;E5&)o);s1A{_rJ^6$iUj#(7?#*zbp7!(`@#4oBOw;X8?tdq)-4C0-{`y z)F37@iIg`cv(@S$aj-|iFYx6yL+l`$30L9=wZnS<{*d_aU%pfEI7 z40qw?wXJ#yCYy+AqBbuxm4JFLQ777Z2MJk^-ugB~S_s&1ZIV$GwZ{4o5Y_-voPR2x zy4I755c1s?ct=#?n}MLE*mNWv8Up#A^tK~7J#_=Z9%PbM1f|G#Pb7+qG^*>K5HEY2 zozfAt7+etFn1Bj#jCOW<5Q6o~M}@xXngk<02@aRx<)Eb%Oo(F_!m6gv(S(B~$}(2M zfRht9ctZKx_X2r#Py9RbA`0Dq${V#64~)yA6DTPJ@MFPKymXt#_~HX+BDw}XP?EOq z=eKbXlZKr*nb0WIuxU3aj+0esQnQnjC)iCK?4c~8J_6Q!lG5Zf@dC8vwW#{FvK5r0K#0Oh9k^d zMJUWNRS6W*k~ZvtS-W5J)zl7f4QHumPxMx?w`?qEJj~rf_Ap8j;XT`r5FYGPyffti{KPy3mE_?+kh>(6k)3v z$s3hBIeSSgbt7|P`5S<@<@JUzeJeZnbe?!`%qsM18#(!I%nE)nE0R+8GVaGFINELiz#LT~9t=~={UM!;L zHRNXC@XzZ!crV;ie!>xI0h=qk)b7i?c0KPX+Tr_~(+jil?c@Px`0hY(cl`JBMr)nu_VIicvCQviT5N~Q8em#Mk zPALf~tR7U`^7?aVk8RCQj_>;0|JoCys-nqin&SJMI{h)gXm7v?t%+`nuX4_uKz81be zieH;;{9S=z9Kac-h!ZnW+T)fh$O&&II&r;(OthXKFBo%vmPV~bvyz5)z!Q?GhlRTT zK{PdjkaL?DV$^xDwrhT9#==~x8Q{sxJuF#!Jz~eP=$bo z1@FC`kyegSb@?{Pe^iZ`l)s2lKFt&`)*j`l1R)c$HsxZJaon#8svl5kwXzBy9Qj1} z6z&S}LPi65Iuu07IDO&>je!0@hnKTHyS&;A8mlXaP$%?v#EaZk!>uc}8g#a%WIfNQ zzn=i6#F2|MYUVwR1+DkMZ;Q-bicRcDm@>{4e$L`+7!CmXx;N2VHm6W~evcDbrG3iu zi_x+(aFgLLxAW$fD?vyvzAHrA{W1PC?y_{S5X2PkSYpbYn_+p;UbpJg|$D@1J`_#3Z!VrzuWoP+m zm0ktcJz*K!|J79j`qWWFJIKzv!es4kfZ3PrE)B-Yi zTDP3Xjrq!s?ehJBm03-SgfyPhF{XnVP$1cJBgLd1srR%@hUCgNF|v;t+60q)Oja(s zo&O%{-7(mA39|eOO+_Q`WUZ|Nk`r>pRVPrapBW}JJxR^fFiO-+Tooq`wHs)M@Z|%+J%Q?RUj0WtU?ziw>sQuum`2%qf_H zEj89M&40ZzZ<1OO8HE*zKM|nAH`?WY%b4+1FvGq-L#^s|N!Io-`azxn|McTM^Xo!e zU&395Bk72JOZsgUoDYy6XG_&tUO1zrUi?HfWY@M1;i-Gi_L!*%e94JCh=Ig-+@yuf z-H79w8{SM*kO#-GLim*0YvdIafd~E}H}{UxNb^;FnwBT%Lrs4`D}wH}3PWm{$y^nY z{zDd%Qr%bnm%ZzOYSVd2!>iyV?o@jliA+Anqy_JDhnIYTuQTtEC(Ym$J=Fy~6|g}L zD8wi)Jz`nvHj-lgediDD4CUPclA5}`V`l-+Ac~t0JdzFs{lS`dj%gz{Q01qjZB2(f zuK%R<_imm6>2?*#RAJ3&luy7Q%ZEn&%8y9@3)f&()%XRK^05R<75D45`E%(V7;d=K z$K0lxS><5G5FdF}+Iz8_vVT&GUsX<662e%#qO$PK4{sf*P9%6lr((v++lX1a=`rDG zKrAZ>Fru&t+B($qbB$?lp_^B`v{(LX%Y`GSt=RixGI6e^aV=pqlpLMy3jPK^7>)`` zH9t^LA%41Xrk)<-|K`Bb1RsUP{YMPKitH2 z&f)|&?|3Y&0s=o>AbGAys^zUXA$!2j8htYN8Z&HY*y z%t!zL*#D1)#Mt70#ej2S~8*C+?j z3SXZxN860N9R9P8kc9w>y50it59WL{OMM2%Cy5{jkh(^482_r@B&W({Y zPgkKFtf*WvcP?gLs!J{4WIP$cm}`%BHhB;V(x~GU1D{cqYq^29T|Uka!yXUvAFAVT zkHo6)7G;3D>)Re8@4{4)Q!OiL{Gl$VCN ztjs}6s|PXtVr4u_{Ye9av4R{{Oj+M{;4v^%w5Nb7JEQ%jE7?M94jv(%;vD{Z%F-?| zSat=wEJOXP50TPYH)&V6X!~y83qd+udMcB;Zlm6n6?HSib&efM`6j3{Pge&G7VyB zUfFO{>ynm^2oC9zWAHBfV$PyQk8_eMmOvS(qr#_*p4my`J*PRuxkM6;5H?t`0gz;p z0c1{V9@@14CLd$V)uEFlry%$r+rX}EkUVv64@c>wHG@obE!+BlH#KXXBjeM1={`0=W?(pV zGbHIUeuL}2fsL(D{=+5T$}wXVM^jpEX>TApeXh6WtZQ#Z^p5u5mbTnilm6K5VueGkkVZ&Y^&1H&jzDu5>l?) z);VFf^cJ`mj(8y#Rd7prW+|*T)&Q%J&Xh-L~@6S2K- zfd3Q%t$xgq+25gF(J}x4{Qrj#m>D>mxc!eeagBTJcqn}JH+P_IN?*tlsg);B2vf2) zIIy#&>DcNldQ^h7oMRNOWl>k(oggO=U1;IP^V^2pV*iaoEORYEbwN_|`n9Fy@g{~9 zYE+kjXRfE6cl#Y7*ZRDp#|4Lx&A^Bjrm57r2=;?#`sS5yW8-5B?)-XQ*RlxG@uB)?ZYZIG}90=ezZhgGtR zA&m4Q?Ym3=!6vS5HT)KKSnA#Q1neeB1Oya<*zV*5>4@^DgS1Ch>|t?A#+GBmdY^oU z8%m5OMgS7&F@P`r&R{D4e=+t>?V*LymS$|*wryv}wr$&Xc5K@=cWm3XZ6}qgr|Wbd zT=doY0dszuYm9ds*gr;)BAuQtdbqkeZY|VI$M!hZ)X>Yt=sto(4Mw34c;u+|n?C$K zfo@OGl7HY(x@YAT59mqzYNLzeLVo<+$;@$p|$O>iV#BQ>> zI=@Tq7({ttq06I9HW^Li_E+y%s%%joJqPe_S!pr-RnBo(Uz1py9{p-ydO8r9?(X|G zJGYt{cbZ=oJ4a6^ndaa?n#^Vhyg*?1L5!$%4u^XB9?@B`cbH%= z?e{|nwnMtO=pkN&{Q*a%WnCc&2>sZ_m0|nKkpxPJLM&fFReG%%O*?#{+WuHtJb8)A zgp{%1Uit8TJhz!_?{%LQaGVoj65bH=t#6nTOKVF@jVxUcZ##*Md00kQPe(_Gi;tsA zwvVBm&S%|Av6F*`3~7+8#uILz&Ih$4$>lPL7q?$;$fK+tCU6 z%KB6L^HSrZc{cDT6b4Zqs(=Zmu?XjEOHbr-?&L(g93A@Qs_6+`1 zUS+g8>bB*`=fi1=AO}`)du)8`l_&SRSMWr`ZCXLW&=)U~;TSH+#&V7OYtM_QGqA zxr9XGH;S_rZJW-tsT8amGjo30eUOco^pA*u3o(+rQP|+T*BH%S`W_#_q=-ok{;jF1 zN)wnaKC7m1A~g9&kG;2UbZC{jmEXvY`zIQ!9YzZKy;y~X26HK|X(Zm|YaRMI?en6$ zW7eP->w>e3xfna~t&OQJBXR^>{;H{bnKc5~KpMab)?v7=ex7uEkNhEcjS=>PU-9{7`mu2Ed+DzCl6|r|^YW58Dn=Noc zQNGD`ezx#s2)dA8edyS&oW1dW*;0FXm8fH-RritX?EjSd(h64PF95(;f}8{F`$`ufj)l0iNCluThmt-tD+BMK_X0ga*q1O7HouO zl-3p^NGkO~W||mME7KpI1)OU@6*G#VATSBhwu5;~PEmH?&JcIVAv*afA?KdL46Go4 z9`?=z^%P}A5J#%$OOWI+&jaxcvisG$y*1G@x&R*c8N^eEK0y{34Tw_BVv?l>=#af< z2ZuEo82o?+ffkh?g{l0dZZ6=hL=l2jUxc><{HBOu7nn>C?;(X{l*RhECd1GTS%l^U zBosm}iH@Yz54ojGbXDA$J{vA(`+Z%(ff zal9QAihv;E<#=P*8Y{*}wWDYgA2CR&QtWcLA1s=7Dl^yJ7MuIoY40Qa6Zq!qp1ApE zCnO%;v$K7U8m#P|u;#WQA>1OvPpO=fbrkC#kT4BJ(6-s;R~-fx6p-?<7(i0_q!inW zj`#`5X?Uoj!ENgJxUo}m5-pBJ94wp|>phPt6tLGcv2X$b7rSWUu^79cBQnKqCct;V z{R8L`5BUOwR;L3dMo`W$Hmd?O;1b|$fLNS*zul?n%0qmu#%~+$fCRqE?DqW#=fPJE zWpz{Rl7%BM0HlJy2fol$+d4+2+tp!7wHp&3ZyoW7HA%`8;x3KJFse+LII_y44NG4_ zUf9tJBh2grkB4@PlX2spo1*i`&u64vq>NTd6iQl>*29tk8NeRvv&fT8=bD4ne&{e? zL3u!uF7i$L`IEOJ5RRyV%kr3`XmOT_f@U_ntPJX?IrL??U!;iHHj`cdGvV9GlkC)Q zfU3CUC{L}(m4@dIenhb?mIWaD^zM$D{{o7yN!Y6=>6GaYL5l4tbqNu7*J zG0;fa-HOS7W~Tqsv(k||D!d{jELb0EJ^6uQwtyZ37C+1lW`(*x4ig+sD7u1Icr3Ed z$^cO<=`0I%SnOPp|5N3*m#sV2{cTM#KmjZ4kHp|T|ByI%Vn}6yYYhcq09~9PSXe|-l(g?#+UWQ$U zZ>WVDBnr!yHDmfNPz+24=yE7=)w>MHK^CnSoGYK~kFd45sMP`UQ`b&yH8Fik zB`QozijJ;b3d(IN_dF#F>4inUprXnIr-fSo){Q^4DDFqP4U|88qI7}tWh;sjrf0J3 zLQp5(o)rE2gxc zn(i>zLiC-{l(3L%3xmSU_<{^ASQ#l$bWgb?xo+ru{7ySwsAnh?3c5rzrz5AX3~a;( zYDiS_hl?+!289Cphngq)Qu&fn3R8F(aP}seqH-Y|+P8&NFuDeMg zX?QdVE9s(PHbnfN<`*LjDnB>z0GxHkX5&msobc8R>CEXnY}w*RKP1z2D`G?mq2K++ zmsh$gl%M3&bC3t{!(d>=UuhUBU{w2RH#*c?*N27f4O0BOO#57I#-RH5NS7_@4uq!R zA{JdMD)UR9xHIfCym0ZIzPA~G1;2T;$>DAeb3$RGOA`TY3`w?NiX}yu?O--$xzELs zelv`%PHmde$XPcn>o?ZTypg1`!yWVbS{R~lKEs>iIns}Ky{T3MYww%;TnbJh5iZF` zilzuHknslWz-(x3;0PoMUj5&y4>d&Ml?$O_lGGaW?&=UTt zXuCY!l^gyeYgy!GHtbI?)(+pRw44~DQ|%a%32btbv;_p}C@fbj{$;$J=DEyX|HKyt z<$TBDQ2&YfHjOV7c5#M*ava%moQ|;|H~!SW(AeqB&pTlVfw+)a(MyuYL!P3A=DvzLkbCC4jA=|H@>#@%G?Q&CX4Q9G?fPW8+^P`)JgsOtTf2^+@kuH` zrFvzyqmmWy>g7s7amg3D7~2MK4>=F&p1%ffiztGw&+e;fZv80|p!o2O-A*;;8Ctp@ zSA!0WaOw?|aFV)X(H3a564b(F{$88S1x4Nj?+9kgKT!11NO=P&?s?H40=NE$2L1MB z%T0aTNQ|L4l$!1sl#Dq)v&r1$D+&<%81&UAuofqhR?PZm;I{c?Rxnl@ulH@;N1XMU z$w1)l3NIELN(ZscY#Nuw(&sx*`|>L3uf6#KLPFOO$zu0uLdcp)wGLTn{; zt|j(~f-}0=>=LsU-^YR5Qxe<-u(_?TI`YmxK6fD=h6e^$&?pvgf5F+;cG&H22l|kanel;|Q=V|Y%%E~I9)HZ6${~n!N>JMB3#3pR zO$#Ox!*w|RMKJ$*Bk{m+JOL;_G1HL8Hc!|GMl#`RFh@4BTOsI|C9uiZYANxWpcku#a5@6ga<#0!gCW_sGv5Gy zJ`j=B&o6r`yO6vsSHr_4^oi|}2w4jqg#?3{llhTX%2qe6aBj2V(3@wVY=$>% zhC#kq^)DIq;Bvnoy$qel=;!*F8bmu@UgBdYu5Ch6>P1wPPNiejvzF`6R8O(B<BGdF+N=FjAmB&p#cDW%;J|C^vn9Fj0NuF04$wkI;h0CS*jgj;I9jPQp*Kw`T8b zuCcq&Teu<&YK5+#BJ5X?_GiU^O}K$wS;Nnv<F5b$|31_TcV#f!0siJX;3LR$L-^`dX0?#*K6^HF-|aXS3z6J711L z+ju?eH%$(Pe@}6RKj{KccQeefWxFtXo2YZ+*3z4eQ+jG>dDFQ~EjMuQMU@6pt{J!H zQE7CQB8md7>0zHUL;+bi_4tavu)|TMoXS2Nz~tldWMdhsZm;nRmukFpFGj}MlV~RP z@zo|pKGA~G!H_;NVh1Q%w!qqMs~c~uqs;R)tJf6q=bFK@MXMYFy~PkZTZ|69X34fO zwfqrD=VbycQtsGmRXj-#ic?!n_8TKJoOfp~O;9y(V~Pw?0YL`mt&M5R5v@spJ(wtg zQ#?de0y8y@6`=*sA)d1@4U-aphGXxWg@jlQ{&9>8Eg{NG(Is;OjwIAHTnp&dRo}nj z#RK}bvpHjiRk6Uqnk*uEn6BlwS)X;Ca6?H^%ZX6qKs=141_XoU74;v#HaKWv0s&u( zw&p8&y2>Q^6oj>c%7eCI%cZ%?^z3{?0wCm<+uvB#q3ngxT}~j5eQ?MK<%5yLW~$d> z*mv?M#yaM1$|$wwl(xYy@J&RwJ+f2;t|Fe}zSIyWM(I_94#Sq6Xqby8m?}UNA9$Uv z`RYrA_sU9m+Fwh5xSD&glbr-t#IO01dIC?CS9eCkg+CDy9FpOkG0#wp(9|NQz8utQ zOjxf3gM&R~kX$%&uI=|A5BcTcG`!d4q~)MNP3x$i*vG|4sdF+oMluHPt+z!Y*e#6e zVXGNGY@qLWUUrY-WD!QkMsD^QfiGTe5=QW8^_w9^E_!?+xn*$S)<>(v7PB_)6kM3)Q}9aT@O7@c91Rq)X}$N|)5U7=oX^ds|z2 znp<-@#-UgL;`wPkuA;zx3S(El2Hgg`7ocwq_op}C;O`-Ts;xn3Er~L)UUW-sGVST> zVg!^fH~QczGr^C2Y_rezEP!h)xsXU;W^nEKVubT6O)z|)K)~b9iubbFEcWAGDR}lpmAfhA5?vt?-nZnB&`VzW=(Xve8huYj z@_cVKG&#vqM48tG0S!s@ep$w&Fjt}uctK+VI(ixh^#||P<2V1wQt6WSJ32q@9R~4& zy-GHW4&X_;-lFOV=gE_6B7efbGFG;#eMOZ*7>Ads_!K#5&V^-R*T^X!5q3PqeAMSv z^vDuhSlxW@HqDnx4gyK7v%J5^ZkE1yA(3zI+DEakBJ0i$Nl_!yC}T#~&3q&3o*X+x z%f3h`aDGM9wz-#6kEvFTPGh^nZsz2*1I+5v9ehft<)E=xUSF|%4DbxcK7cz$9>mz@ z&eKGzx(JY9t8MLW$(SC6Mpj?FvvZk`q5;!)G2uNkY}*(*-d*~U_W*hf$7<;ZUFP=5 z9`E~Rc`$Yy3FUYLJaOKeln3UD22?bc==j1OVe01eFDMb*gG^RWocMJa$!4_S`bQM^@)cfm2{;CUi27U!Y$yS=tBK- z7<%FZDkrLZ=$A?W#?g(7{w>c2?}pm{tETGz&3ISK5`A4x)h?q|5|N5un20dNDjsund-KJLsn5|mw9r%{o=jf)g8yELU(x4X%7t-fXe;qwynp}X>gNVIEpf>9=e2}yntgJn z6<79Ol4~UXMR+*<09m_(O1t~VjP*{E>_#xcf7KD@+WRG^-~}alWm#JTQ)i}t z@=^KdX%#iEpYC3F;_4zW;M*g6QFh z34%T@jR!NEjDlu18k@xYjHs)a(dEyf!$)yO>=| zCGKf{j?iDXlugV7pw0ci)`}CP7u3Y0(ak*R^|0s^D@tGwHbi!5PXO4%65g>1-c@A1 zG-$oLfc}n}U>dOPV6a5lPg4>#P~K@H$D8Oh4JAngS4uE7tzXm4B9dnQU!w$gh z&hn*cPS1N-tKDmLqv>cQg6X)+=y&oy=OT>ld^yqvmoFbK6KPV_j+V|aK#64)2y7`2 zN^5adoyU3|;A};hD%JY1=2KKz!i3h2$1!UGDKdY>>6{uv2H;OS+l8O|>xSsno+Z48 zuCP8}YS2Pnc$KL!7zP*9_)5RILmtMGo}9F+>eFtZ?~zaDP&xb^S5=u|{&JjDQaiUg z8TuzMvCB)8at{FimtPx8MNkBaYG(EV<^5F?IDax3C{LqLC?hrL#Yvqdc&VxU0He8u zca%?udQ5_dyf8`FTfYq4Laa)=X-e)fXp+|^ID}5&fRnKcvR2Lza*{v}p3wr>K0soKW2DhA!`VeFmFf-Tz2;_-mFlUqCa18E1CU4 zUzj@7%vZ1-`vj^WOF5n+Z>N2*Wwkf7fqZlI5CIo88I(yx?bkATN`>cn<{5;9WmjN@ z9asWHJwdeUZE*yXGCXN)oeKMtCX1#RN`m!J5M$WpeHxz`5Z%!eLegf0@BCx%o~J>C z{m6^YTf4lDY&DjHwZo{k4f~pEwtjVI60}MrNeQ0n81uVPxnD%vzF|X#no5lz7WxfZ z5W$y|+pLHY#Lql9Gf6>5NyPPO&Y)^gQK66X9Z1ywNx3Fw57-Qh5+-{`oGl(spgc;W z^(SLmp8x|vWJieGse}VD#)FqbNWv&rY>LpNWOWbM@fa6_#EOVE;buUgB}Y3()6`(C zBmw1;-AT_TS{n9waxqjMBUHPXSP88UgEY(>sjEoB3>%MPNvAiT8YeS~)8Q2-sg~53 zsMravWs&f4_jSV$*-3V~9}!p4dWpS&i049kdVv}Mg3qe375tgVN4<{;YQ*?=FiF7d zgp;i#Bc{JcN%RT%pZ|rky{HA={&KEtVg8#d?*GV~%}uQB|F6V(E2YnNg8?S=#si9| zV^vr`Z^$1A*byTXt;4$Gl8%AxfFhYzFd?x`j$gCo!c-yxXL+#NHo;f%&`j}u{v0S+ zUA4P`1Z1?|{34&qq=Feq(zBs#6UKh8seiZI7qi@g3quC$Eyr3-h zu)b%8FrKlopMoo&3z~=*l62Xi5=g>=Ef}V=sD97y-m!(86nOhV4H>^WG-t^2ebm!W zUwCAVz`jRBCrdyV%r?p!7Bk!@wK%=A%VU@+n_3Tv*iy9+v5r5%1cT=04zutH%7t8q zDwgz$%24PLbTX40o^_a)IVuBn6u5n@adC@BUCry?PMVU#1G($+tJXjT+i!#kEkXiZ zyPx?p{I?FS%T=R-kRPyF{Q#`8CDQ(ixKLpu2 zBvSafg70+iTiRe|7ILc0ap#HC$k1W$@E^{o>1W{nM7+BbB8A6xpcR? zdtKk;(NN|LJ|E|}17@%PB@d`{_1~f+7y!T;#sAzX))r3A|F=`7G;I2II(CBnam9&i+}|Uyng_?T@QTBLL#({(Myq#yCwHF-UHj zurZHg$cusi4~_`lWD^nNE8YR@-M&r#$|F;-t$;WSUbLuHgCIX-7{$=0_8iIKK}gdO zL@@K89io86_YBT6Mnql2RA-@K5kvAb4D=8zxyz6R|mm~C)xI6bG zrz4QY6NF|e9o%_sb%&vK1^&e>mBp!qvo!R6)$9Q6e3LN+=#+EbcJF*5r8*p3eXC$c zRKNuKvP#xJ=S7Zn)G%EOSQ*STzh+uO2B%;VU-G(u$$pXpSMpuzV;T*IaYeuHc$2Za zts!00o@9Iex*Ud3_yoVc!)UMjocmI6*+XMgHOQynIp^pn|<`QUZ7 zX>YzziAEh@-7NPGnACGa)xDdM$IlMY?jG(0;hS>}T|rF*5~La#Bhw@Ym%qb0Sy9?(oE^N5&l02g9TS?% z#Rp%FSN~2-Txz#8(3EJfFOs!n<0BU$qm+yfSQgT!S#tY#cdvbdx!ZyG~BgoT;c=u%w`^)#JPFB4A|!;j7v+|T{2#e z`UN3XA1*l|WmCjHwiJBzv%(HrU8U+rahsop+u0 zgOF-<(3;CmOzuR1AacD~v%w70PlE?HhdKk)$+!sX*9?7WOdk?Or*yNLPuAAX!oN0_ z4Z)jJEBUr7hua2igT8uM`0Ol@s^-x25J*HvHGeci-&9HVPqE>Uh$hl3b6WPJUY&xM zuZx<#Zy?ZEGCZD}Rgrd3E5o6Dv#q@ed%UrpWrxR) z-ndu~e%TTChJvw<{#sQ3!U@Y77y^aHcBdy5*r}JeE^Jmr|7g z8lRM}ruFa@ZBk48&nYGew7%T8EE9&G!nbA5D5q^$ry4b7l%zt-pk3SEw!Q)JimDw_ z8#1Xo^0e>Lv>o=;Sgkt(8FY3Q-fbq0ZA&Y&o>8iBNo+kL8Rk}c*Vm}vq#0njxZrF8 zS(+pnY6+V1Ler*;1}S9}lVLC|lEHi_2ugV4ZB-k3=xjwhgm~!lFZNX*oxvPSnLK_? z)bt_4_7kq2=gF4=;z)E_=ZhHnW15!Fh|$st)hpRQAxq9=9%t8n1Cx}9PN7G|R%oux zh6?zTGqDkte3~mZ%Gt20pyhh1H>od*I?J9^D*03U>&V@_T?EY*6Ak(7b`!OYYoB5x zGPhEvn-x3X-HOz0#7V zUA45Imz>|RU1o`kH8lD@)h>JJ4F;H|{#6JGEo7;|p?A?o?O-Ad*Jq5$~)7lohnKC0otulEq^*TD5ZNqub${^L?Jq@riF z!GPfRQaam-jve(k`-*h3ZmtG9ND%*DDN`^`E;e^Vv4mrz^<&44G52?35v@rbq~y`Z zN5@AtUCjj-I#gOfRCtEApyW0fiS8fh5sw+HMnp>y#ga)d4rQ;A^%F_u;Q3C*Xdp=m z1&MqZi*rL-ro2vuJd71UgL1S^MgeZ<5lq@>!014>q|F}6frKq)&oB;GYrpVUP?KCP z1%dYoR;4PS$vpCDwzQm(JXPoL15S`wHg&L&+^Pf|io^s-(&aqFGE+|ZE(>WgS5{v~ zfHUrxL_yi>-q{F=8NFr&6#XUEw<{($kpllV%J0|#(CuY+C%<<1)3#%2tNVP$eWAsd zyh+&pJ2;oH=Zip#Q=@ejtV#AWi`{nA=Mmq~1d>>zm(TXQ{GhQWkCyK{94?YDSu*mj zCk&y|3pg*wLabXKvtCR8a;eKj#?AWZp!>3VB^d^6B7u2EF4Gr#BP3s znDQU`xu4LY7UvHN=L`Hes{S7LgrZT%u=Pr!ToMVdtHpz}KmzZLnRFub2x|!iqs8rm zMQE2+EfNHJMqSZHMn76DVbREXU$rH3S{&cnWI-@5B z>_&&;&mryBVPv8Prn9j)>$-5HY8(md)+FCQd7x}+}q1VUd6pHoLX zD;`kpczV4Y1CJWvi3f|+g{B#{v%9VU=E%96)^M@Jl|w~;Tks5rD(S>*`Hkmna|OUY ziRA*g0sDQ2>ObR(-L1OIo7UmBMPIBm=iC<&5nGc-I0u-Sd?7@pQ=9`d7%dfy<(F@eKtfi&r@p zb&b_1jGuu2`NqG1B}ZZQyEdMFvEcvH+Hf-Yk89kPs!n940E*9xn)YR+P#mk)dmTa3 z7A9e-uEeC|WaVedKK?-Ml6fjI~tx)P+gYAZi3Cv*tlb^=o>4EVubDM-q=UWe``I zehrlwoHRISohtzR6Jx>?kDY2B<84jL zy{anpylcNoP>S!NYpXd$=Ng;11zhMQJ1Pd%F`PACYcvW;tP({3!0vK*(M>?&9B;LJ zGdl^UItqajdL!Q z{+y$V?Fsr48-0PCc>{RE7u*EI0?TdbH$bG<=(4fwCtj?-ksl7?i3Jn(iE!`OW8?7* z3j@`q>6d_6&RtLD%pc_X&3puGcDhg>`_ISGmZcb{{jJ$q{;I_S^3sPs8B&TeDD41y zkSRwTKD_G)L0ucIn7oasT=BUg=Z*gA4LtH!Il24XoK*gon__9XqRv1`O#yZBbfD7Z z;s#E=my1nP`<{Bt+Sec>2;st+n zvi2^4P1>&)Is~;VIl4G9Hohq9x(Uw3prke_^VEZdk^9w5UhlsNKRXAn+U>f0Cq1`N zQZMj1B)iZGhPse@H`Dvy=)?Im@Bg0!lK%~)2i1^NO#G!TGyJ|t|EK(X#)%}VBkqX*(M@-eIZ#SM@R1)0|XwE=1hx+v`+NT1n(S3@PJhMC_2bukt<8! zBqRmZbq+=9J{LIUf-Hl1vS692lxE;{Nsww&4JmS}LQZw+#U?fU7H+{8iwlXRtE#(f z;0skG)n}E=#;xs%I?IZDq~{fTXP6ok*NzQoikK_M{dama&IVg@z+(8)Ta zS;$P50yLbdeErePd)Wl0>3qkZRB(Yk`pCGYFS2-f*8YKmT&6g`Cq)&BJUMN1bf&^*TYX22 z^Z8vtZQI7T&%p7neS-E@&nH&?6 z!5Qy))%N`h_YeVGljAM&O@yQ93G1&<*tpG^xoRcxaSkk#al3qlg7-@MjGF8T9N5%g zfSAU3Q=|mvg*M&X=n<$sQ>tI_CnQ)^Bf6#a!zCJ*|B@rvukmsF!5`G#5B zRUd8t{igmxA7DszyK&4+cxfyuK-Lo4=&uwkj*Q;d{;9lPi$f=9r(jbq47=@lxsf=R z?W{gF0ULLH=37_SB;Js{tNiGi`f-7211Fw#D|2h|xm6SVo34KCNvz|q=T#h4Qc6@E zHKzGZpZJ4RZzrJ(=xRAe{OVU-V-i+kXd>dzDK0?(tRe%YZ-C#YdNfnAam{Q=lUwy7 zbnpRlew|M(%L?-`hmXBC4!BnIG$8YggD(EL%SAvp5=UHZ%g+$_xcARzDrp^?S}9dHgquyp4&Couh}IwTZ$1 zixAK`bNt`i*-q(fFh3j!yu<`&BBrExK8{Q&Bzx3|OVaS}Y9IJ-Y<%TFiucgC2s8(< zjkQ!qWwL9}Ov2H&K&#!hXbOAFR;1YJ)amInu%oHnz85pcRp_hP>FUX7&J-_y)Xb#x zN@wpk9>14C{e9yo8P1_y+IuJ&0{_FpORxi=mkGRG^1LBY0>!AW#*y{Fn^_=PqG zVQ&jML&0^tz!~Va`zRfA!tbNS$0di*B&8Rsmgg?F~WZ}>|oHS?>|ufhTLTG8RN zUF$_8F?`V(aPr}^wcRgp&vMFiLryl=0sjc8%=tpKv;s6%X|_XWL|I<7tay`zW3X~j ztV3nZ&$F-~^x?>X*0ABTy;Bcp4JB(I<~(?9>yVa{7WrL$o|y?!J=n;PzT0}#w}3BT z?AKDxOm2?ldmQc*4Lvo!I@8%g@v9Wzd~BgFELTkjW0)%k%TFdtl(~qp)xb?H%3) zUiDk}If9Wix$P`ppT@HVLN2>znZw7|<;j+-nP`ue(`&LrYnjj&k=4=>8UFN1jLjSv zTQ^O85^P~*qmrw6`xIu_u<$U$iBnK!1uE)g%qPTy>XyLGPLG%z_^C02kXTTo*%u-f z?CivWTJ7wq2fkq1)B6A+7hgeG;kP|V@rSUc?`z4*#Y?WDpliKCYy#Pg0=q0d7D{KY zL91gFi-pZ4h`S1>13VO^Q#RVdDPpAPGcS7OXH)FJsM{Ql_BThiv<13#-`i}`lxG4a z);siGPDpe%W5&Y4->ZJ{BlL9mZcnx*_zYuCQv`8~;zH6LqX%(nSo=y`tWY$4GawHl z$7pD33*etPE3Q_U4CnoJ4xeb1AgRm9HjF94UJ_!>CT}IVMYe;6H;iwc z?`Cl-*y|a2NpDGEANy}q(d=b{_D+Q&Ea%$=LunzGDd(aV@Z&(u(SMLlA;a(sf9+Er zMIw{JSm zU|6+QMLlF_FQR}$wiqPM*AB^~-W;d3`VEbb$Ii2l2O-*dMN zg($$)Xt5KCITKcsu8Y-yCSDksjL*3l(04+!w>kdukH?rWJL?CY2Js2-Ug+vw@5Ug> zQ*GrL2^jwkNezzaTZG7{7`l`v7vp6H*ok1oGF3{!qU$ zh?)=X%xueiKbjZ+G-N`@`-6>rjTgy}*WIbk)h#9Dm3S*V!W4Oz6u_qZOz4F@s7Pz0 zYAp}$;)FtP8_p=#X3Z|v*_`zi%;IX&?Q>zPq@D>ReMk9ZSgMguKK^GI=}{jm#E40g zWCTHls!i*X?l$GH%&?2gIRk$Bq4y_>zHV+y0)56gR@2F8fP&Rr*GSxZ855ARP9RLZ z=@zli?gL$f8*~RzA4EYIr{LOh{T>+0Tm^oRl$gW)FFFB7p+i5+W{&Lke&@3C9`-4K zM^$g3Gn(WzlNV$rvRitTN-I;r{oFV-gu8U{EpWi9?qFcoC7 zi7izpsu`RFT!6|3LiE9(g?qq6Axj(m;R2LF6 zOI0PZ84U_zsgD$8^~J*}Q_dBJ69bCYoa(qb&xscW)AyS{_A1^U?k=zMwy6M7Ni-z# zbj}KJK5^m)A#`dE^>B#@`8EcgGbl8s%rFB&s4eTbpquBMy8b`>w}!R{-xcwf74ee= zg>m034YX*;8GzY!!1&32-2n+zAexoyX^V04{+n5y7r+A!9*C2x;7wWe!}Tt?e_AC^ zs-vcIp}8$M4Np2u)kdphwl~CXljXAp7P6D`c<2d%ujQ*@rc?WEKBk$Z(T}g~qDnRd zlzlJCA}UUC!cs9hB!+1bO#dB_{3&iKqHeS0X9vw>Zx(ljz9L5}b~y%(D%n*x24_Gw z8-Ssjj73)=m12bs+$gybxo#7f9t^TLk)jalfR`!qFMeoLrlOfg>g$w4(YEhN*kQa9 z_GdJn?rlNvKyIHR@r&a-;4)bDib{SOM_!b=R*GOM#_Fw3LMOA@>_E9)mEOg*#&Jg# zbVFB}S3mNfSj6r%2TeT)BnWrn4=yti1r$x?zzeLWXX>AXe55S!2C_(1+Qhr@D4=9ic7)6ibUMZi zKqHZOjV5K{(2G#5(t3@X!f1jYw=TGJ)i$hZF(#*uI;ryGd?FPBP{7&zM~9;1>u3hy zs$ad6OubHqt!g;9bs{j8VRtgTo1CFqsctl&$(@o9Z*Q;ICjM@PPLp74_QW*?vlV(5 zdb*}5DGqz(MKv3`Lv1YCl?~lh&!(C*yNOyNSfE^$H6+}mwp-Du2gysWQwML`Y|kd(7WNUYupT$<1kTHHv@jr=u9PEqLq~QK@eJZH-Y;z%El?sR5e?-0 z)5-l3@s8Xd0gO4Hcz0M-F4N>pd+$6L;M_1(xf;E*rKFJzDjC(Xpw6YB*KS^}lklbY zSwnaV3-1x^UuMzu zW2L^LnBw|TywYnGI;@DU-_ci)o7&K?p|YR59X!GQBPd4-IRE}RsspNn8HLSsw0dXy z8b)Iu)pqVRERJl;gmrq6!|BT5Sbxcy(qyoSf;0A;)oUJ=@ksY?s(a?qN>lY+lS_SN zrGl`F+zvl9Zh5sfYfPSseXz~Y1gqIrmk0NH9Qj7)myU-Y7`$79lpQuhK2Q?pk8NPu z5Sj6%vhdVXoYE2i>!WrAtVa=znAGa03IvekM|A`lNo0$bOxGx(mX$oYm3hY>kANg{ z1#vesP(15xplCp2kYz=Vj6Z-rBWg@O@C!d4nB~4me?umpgvY_D=}T@SFYTH4dM@J6 zr~o263GRj&inBdJPznUjgQn_q0dHPZzOt*KI$XFExTZ zjlaN+>3gu~^!#aq45?%hitetK-BL!OrW4-TS!QbD;oh8P_~#17z;*VIG#%f<28F3> zOv+I(WiFqs~DDT z*^MBE_#~3X65x*LigatdGKofAkTMw@{V^YTO&yy_k=zaV+} zdAUgs3(NWk+o-uv=%fV%+X@vogc4ZB33ziN z`TsO|)~}p9yXMKE%mR+b9Nm^_d9|kl|7Ct@2#xJh`PCTJ#w-_XgJ$$;@jh? zeON7Nd@Ri$gWX<(d5ilceqX7Z_WHA_rFK%svA(Lm)6bX3d#1L!qje8J+|nG@o!Psb zzOe1%%1>*+cmQ+QH*0C+U6c~IzIE#_tYI^L4RIdm8b8qM?HnU({1BKcEdPLl183Ol zLX~sM6<9Bw*f*hl{+GO}W}~Bg`Cr*b*>8^$|7Qfk|4xe3`wzOEiMxS~y|u|N$2D1A zMrz;}(Rfpf;V8+N)It>^)Vm(ED+t>4muA7(kX^=g;If4EXM3s?55+m=VY}1g22_y1 zRr-_yw9r6{v1%Ey3T+|;fnB{LXb<)>ek%Zq3VROE*jlnIqtZL)0P?7#htCMOq6&t) z?WLo`7qt&A(lB=HF8%e-XK)lLry+H)I8+l_i$z5i6Bo-z783gg#BHJ?Sh+`4cs(#J zss;#E;BQder#?&bpn)=&m^|9LOUD-$nOY+lD0-Y60P3^Z`jKbzKiw+&RH?bX>^{2b zKX%OTSmNB!jGbUQkdFaZ%hO&xp+W3~os^W#a@3@Qj3A0BWlc$fwXCjaeznwD+a$s^ z^R8WEmQ-}#)pPc27t>Sv{ZevU&*3!_UF4qnuY50@ZSlzp>r*x#a%{QoEj({Dv&V!Lby7+`|k zctQ8ZGv@M?2$c;lAVy`CGjFl zAw-(Uz7%<>{GUNJ!!*DD^Z7iVnd|d$zH`sH_dfUDbDwh@xANEMYv)+$Y^2p$p6Tzz zzw=zyX7da17dtT{bB+A$8#YZd!aCUS`&HsaRf|BNu`` zw!X)=-_`)s^cZNn{Mb~5JdtAO21ND#-H3g3dz{+OQ#9}L)k#ZHcrTPrD{>bV-KC=c zR78^}xv2DrpH5Y4p|ygkE)zdL*%%77x2G?V`SfBpweDNjSdNVBI;GNx^M~oz{W(Jp z#PMd`0u?v@a)@(yGqFa-NN-A`EYMi{m}yc0E@8#UD^~Qb&bX;VQ8(mPZ_n;9F5?)fOaVC#GPFI#cx`q3j>BlpiL)dw;xX{o@k5g;SlejLTZTXApf9uTX zR>_;DXqW@6@`(@x62N8^FFP9_uWOFJb|_})HPA%HTWSH~ElF|c*;R%QsmXa5;}gA# z1`CGdvt+wG@LKBfZNC(`7q?uvC^lbp`eGA;>bmS|%YWWH&uF*k!*v=7?v(C4Z6AcD zkd^D?|?F{EX`Fzh%4?sYAb$WjWe1y zCs0EXTKI(0xS)n*y>Q-QN%GyTdio!ng*gpg`~faZ6QP@r>f%k{Txz>)se!H~*-&`~ zILqgor5yLrzw_1nk zpZM9jWX6U>aB()$Ix#SSq_5tc%wOr)_ptI-&TZEpHJcf1 z(E|~V*%tpgl!m7dfnI(VD9+qN*Fhfw*0}=2Bs)iUEELhc4z4-c1E~pqIlPvQr;rY> z;C_fzx4KAGq^fMLDy;0p$u=>^kHjoJ>XUzEF+_}-jNRy$>LZs?HAy-#+FmB-y$|HTbE8YQ4_v2Ao#vi$>iKkfe!v8F%FIG z$O)1@5!K2%4=;8}+nC!TGnL$Vd`~#(lHuHUwU#y2c-3?^wpRoj*vRqSBMSpJj4zrK zwq1Vpo))R-EH_v-oJcHfCF ze9KJOsNZD|YO8T0(fm0#r|PZbyAcL8atmz4cM7FtErRo7*3~@_1v7{pDMN_!3!YQe z21?{L_~W`8U+thnEP3r`uh!z`bVNxveCigvNV@;fxH&q=-$mY#5Rxt2>V^pa4q2j6`> zedU5)?~4I={SC?SgOAj5en|0%`X`$R=I$k4J>4>ExNy64T0QbpYAm)PY)YwVgfL%*Z<@JCT%6jJQ`IY7m<8T0Uv~Z{p%CF5^Jg`l?)Rz?sLQ= zFDKD*{}huqR&QDQzBxHRfre44aKm*~s+l^Uy*G<*V=%G_+A?QT5D&T9zO$_*QVx?0 z4O9ppE!)up5qDT<4@0k*R*+lTLLDaCI^j`^DMDOtL`4hdPA2Y#>bVQcdbeDThPmh6 zk5<2b$^o}{deC%QrUN!%@+Hkr{cU_qS#dcRB*0wU;f=&&7`4VnMqZW=9dpV7ri0FK z$I@Ion(|=lXR41B*{;+=T4ES`t_^!UT!z;Z&eF>aR;RlaGwu=cJdb>OzCy{91s%2#4kz7sJ}s+{8S>GRRxV>E-1wIaX* zGYj~(9_i1=A7uSA#4y^fTAOy%mlRAT_OuF^^7sRO@gu|Kkx`-r2&N94({9-{qL}Y^ zIgzX7MN5KsRm=0(Icr6a=C-wHsq_mkj_ao9ms+)>zqyXY0()Gvsb5s&*AtWar-Qu7 zI{!Bvbp2wp2Vt3r_Zz`s3F*C$P!X4tyl*{iJaX_ zJ@xVvGA8w45bB%rUd>r8F$9xJ-O48#r^p3*HWXQvq=$Wdp|Z_2{5skp&J^6t)~1#X zhSdr30g_v)wAP?&{>}bQrt44l*n`-_a_h9%SX2=O@He9i1TaVPamr*7)`)v;0v|$@ zbPyMUn8Uk9L=5dX|Fh0EPJT$74m% z;TSni&vMx@sKMLJ0fb*f_#@wk z>+*AJGx9|XSI(SKR=Z#RcV^g&;OLU7b2+%ubR*6)d!tX{U!YtCIJ96_xkq$MHgC2_XAuzsQ%iTUW~;wOBf zcHK*7zCTWqk^dAZa7EH3b)n;ZQQ2w9Touy*9fzKoPQRKk+eP|}p#fX@J<>fJyz@FQ zXS_gmB5vd0q&cut0$j)wAg4pN321=t$omX%BcFsq$i0$lLJ)|fo1-@b0<3=6z&h8( z)W8P7-@!USAa*NzI}+r>^*|1-sE+EnaUh}`@c+o0{HGis3G+2acQ-%)1akNt0s+f` zxy}m;pBEJ4+LwVO6M?v6YgE`R@kR}Rk~?ry{FH(6D>Uy2lZ2e7z~2q*xC!Gm{o68y z`JApGl~|4$RofKbKNp@%j%DL4i}pE8K52?=gDbIbaGbScJ-JTR`ze+E@=J}{ku1l> z-ppI~KuR~h{ab1^r0Wm(ka2MT-mg8Nk4#-v0Q>7A*k2tYi^yMty!N$20s`+Ba6q)6 zwa;6=Fz!K|pP!yEv6q2B`M21Es=eu^kIyCmfyjW-gyNT2q?}(n0>}XO9`0Vq6z8Y} zP_$odLe&5ct_|rtQu5C-DgkN8fUR={Dbyg4pApiS$Ym|^0t5F7z}^4h2idU?l4BpG zcpDFNGY;agR~}Lh%SS2eyK(;Cfq~H)IP6J=MyUm6RRn@e+T4|+L!+;fz;6|94hM49(F~oR;C5)0OLCDB@kdhR7DK|+S*4P zBpq;faj~=U2HNknqol*iIB1?cFv%a3#rn(zKT-n(3OB?#ViG(-RMKG#%s)`Bdt;{{ zcooXJfG*b{aCcQLJI>pEfl8?6m?ks-KvgvI?`VNWTy*EI0PIFNvIv;RnbA$`}t{g=fbhB zhwK=S++PO22hO-T0Xq6^?+`$EA-ZsDJ2x8#S1T`P;iJ_!wC`H@VKx3>?*ayHjnBXW z%K)G;{i+6%)Rcl&4eRUPb`T(OlcSsc&*OEdH3J>Wry9o2n=q-@wF7KY0}PsAG~PlQ z8t;G~%yLij5}5N`|%npy$0prHz-P(T-%n!lX+> z59}=T3_7T+?~l<~`(p!IJ{?Rr`!wu0^o%E{xaR8PxL-TOhfWMkME|GQiRclzQHgOa z$B8KMxiP^qPm*J`9D3AWR9ITuaoCX)rg zojr8kF()M^9P~YQ9J-eY6_<&NgW5`v4F;3X2@}+hh!U#-(LE`sP(J+s4MqA@Fi|6g zuv5{U5~x%|0xVPvE(%QaG8lF``sO<-edz=iI;O37O!%fk?0EE@TvR+0@p1g`eO*kf z$qno*^c_=F7Awhd){&hNOu!X7Dy;1ZecuHYs7Zzf{C}`m;xV!QwDLt|Nl>D(P*$Lr z>NI76Y0mVgmGMsw7??BN_{sUbPDTTKdxW^mHHgzEaR@jp|DHW#S~jBs+J6H6RE5JO d)4I&jB@+Sg@;TBu$__dWOq-RdfLSf*{{W)JaJB#d diff --git a/python/lib/py4j-0.10.6-src.zip b/python/lib/py4j-0.10.6-src.zip new file mode 100644 index 0000000000000000000000000000000000000000..2f8edcc0c0b886669460642aa650a1b642367439 GIT binary patch literal 80352 zcmafZ1B@`;wq@J4ZM*wx+qP}nwr$(CZQC}!wmtuSnLBspB`;ISE+lnMcBQhk0v&QGOsvq7S?-^UssrKY{S?;d*)&wieEMdUW<4|230^y}AYwy4HW6Nl6u= zoaJ5-LR@-QR$A^vl6rDZB|J`!nwFAQGA2%Ke42Kgo=QPnnre2Al2#%n?2d5qNx#vg zLS!W4-9uaZ{$8F;zKm8{9bO_C`oE>MI*Aq63km=LhxLD@WoTezWpC%`{QpU7MO`Xx ziw&XoQw?Flz@#5B5@*v;0sWj>WS)p39@G;8D1fR(qEen@k{AUt3$zi z;{dihzth9IJM2krlN>HW3%ZU@f)zh#x}qz1I%iKBe-(PUFCT?dhT0aD|MeuiGxy4R>IH0`W_ZMf4)j+w+$g z&DpJ|DXf0EK#%-2Xw7m&$!gLmp+z{0+|j)9=N-Q-kysDt#|9;^#5;m^kYh*zh&4#| z4df7CH$H;4FfyEB7=l^*M|dV=3w8Kc42583#*=jtV%zL*ueu&U2Oo z&)ds{ur9liY+7?~5wuIlOP?FV65|eTeC@3ZEeknNdgMEe>Yr-~rnGgV#w^%xwuzI{{u3=pNbvv`QwzoNZN_kuvWq3t?@y$ z8MEXqc4>Y2dgo}^sFQb_jN``W;U}$r=9)Y3S(74`h~W{;EmYZKnY^q{u~4q{4VQf~ z79KOih$5^^(v5a|aY3h~Q(t8JFvuJMhg%~`X$_cd@_nb7q3Pt3mWyq?5X=1Cj-+d>H*(Nq>;;*OGB2~ z6}$*}Tnw0Z$!e_-f+JI8IeHLiZWRdIV|J<#VaX-l9)ZP_)z(gWnlBX7X-&o)PRc&< zKOTvBXrbPmD>-L= z`cZbtSy-iQOV||H$!Y_4TiFHsY&F^43qL=mKdxUEPMtU`d(E{95DdDFx%`kp<6>h6 z#S4|1euLb|Jb~|)-B;LQ%_k-gCC-eYUZluQgEXN2=B+z*FU4ld<*LphdGKbhOaGWD zJ*gL?wLTtS-$re89X)Dv*)w3Y=s!~ySv#&?=eaJ+n$M`ks-Q7b-^}BK#+ELLM*q4? zzks;u+q55ct(g4>p8kby<~YP%=coVxzPbPaDF22hBWnv2TW2Q|M^_Wae`CSF!0uo1 z-@xu#`y%P6HRZRq4`{)gA;nzTHCwqvc|~jGVl{f}!l+g2NTZH@q<}!kx?S`GNk;m;nclK2jw~ zLSc`rMZB#`+SWu@jR>*IxhIv>J77;TLtm|dR+R}ET4PTt10Lz#=z$QCuf$}BB%@tC zwatRA;3~_2F&aaf+Te*<7ur#?_x}v1l{r2iA|ziF@?!8en1 z0+yUUz;*&d6itxsh^nKNH%F=JwVFeo={PWkG99)Ej#hIr`S{v^z7Coe;@DNGtt95y z_)cN9NK%NtQ!>4}8q1?h5+aHZ`(SP9qpoAG zp|pO?h_NlcHeZ^L?@WKo@Xbs#0MhQ~q!?NYyOrcBY0sUy6RXp(QPdlD0?Ed>-U7(u zFB51^`qfQV+$imO-#`mYU4RnMoxcI>XkFPQ%(Bm#W;-?yH)njvkwS zb9KgUYiobmy1eW{%z_!8F=R9wi)_mqCoE8czP@cTC(LyM2ypy#DnqSW@i#M#x|%?EQ7L~}E!QUlDq)X4OqO7?x5 z9sCQHB{GA$z}DKWKFje3nu5Bv1+45-_3_2QJ*$^sIYjTKZZ~4;-}~~q*MQ$GjcOuX zC^TyH{mNsx$Nb4$j6J{;SX>b6lrm~!&7X~;@V+H~ZXsNRS;Ha*t1VXq#P^*s%|wt$ z6jU%>%TLBW&Kt)Yd~#`+FzrrSDP9@Xsh-AgS8)uRU={?oBCedNqePLlNcqXYzi$i7 z%2}`ZKc@EncP;Tu#(PgSNT<9E#W>*dMSoZQ^s-Rwp5Kd-h z-XWRydEzVVYG!ZK+8jO5_s~>wf+FX$_m29Zu+IP3U}LytElYKIfk-2acnT}62p3uw zv+t9RQ|gpS$?JU7w*C>bke}#Rn1cG=88$+ygmnt{ei-`8s9^}}LbWE5gb4F~l-KwB zCTSRa<462Kdc<>S(4K39~X>J1BqRJETL>by{2Hkqof4uyPm) z24-Q)&~AJc56Z?}XSFQG=bP#Vi7e;ErG}p2mu;LG@Dww*L zCwP;<%9KE8qz+f3AyB#6w(TMnJ|Vz?d9)(SU(op?8ead#DMy{;eYsSc;1F#x*lJ!C z2CR~rVu}{{HT-vnv;YQ6^+2Mb)9I^oz?f7b z17Iqfu6OUnRtXUWj{WcV6cbBNHwjTYWK4sh3atk=sFHQg>8KWmWO+&(HL#RK3nFJ6 z3(tj#_Yo-2Fa3HCuSEm&)CgMyfi%WHX0c?aqR$A)zb(S!ERUswrmB=4hb5?~Hkl0b;BN8)z0 zHg^Qs;|E>fZn)76JSB~DVuNgTmUfB#IqMeh z6vxngU2+c`kU4|#I`@W-iepJl3|@sN7P8zX-Pnj;akhHD2ZWwKWKU96v#dWF^cItU z%aS4PI>Gk_?AwvoAcbdU;83eb{q;5{LZ+Yez-lM#BO77aSw>%gSVP*LiSa=?`OhR6 z>Yx32=Jyj2=2ch8f%usJ%=l9_%I0$VyUwJ_TtFHV5AYsaL}fn%T zb7qzq5c42X49xd&DK~U-tqf0iQ@q`hQLK|iRzYzP!`ts*_GvTSFZ zH{BpjK;lBcWc4L{l?r4WXN~BNrG=KdD5fR;-mGYN8(J3Gr zt5B8|oMeIkWsEorBX$M%VcnsBgpR%0G_M0cSuf;tj*0v3B?XpXt zG}z7a+yG4#ehaCMCx%&G9LInwL!GSXQk3jY&)B|SKZEyM4ed6!dF|xK!SDgXrIBo9 zT$|HJa>vLydtmJHo^b?$>XSxd-4@FB}`L(=h^|tk}qSaat#V(=3aGd`WDp zy0f^*mHCrN0Ws>ML5{1X-EdZE7(sRG!86)6_!Cbtp+4iIeshH22(O&+#jA5)w4=V? zyBT#9VR+shhEWFwY1d8Ak#Cqfwy9yiYP(7%>IY9t;gjw){ukOIK1A&rr=9yx zBsm7GZiWRfdF^V7YaB!)d?i{Oem8?C2WirE z-_9scgUE^|FZ6M_n;rS}e#^bD;d4~B3jXFGye&AQSKut`<)c85S2i8nXB11FW%WU! zt|b#aI^_B|26Bohn`1G&kfxc{->1sIgOeH^uiU+YoJ(k>C+#*C->J)Afj^JXH1e#~ zTz~$kh4P+q44JH6tVhl<%&#yU@7t2mi=3FL2%5R@+;yr$s%pz{lWkhmD1D^J&+ww8 z-I6|{nD->^BDl%8SZM`#NI~_}IGeHAy!@LwVmS^6>nGe z3@cH_wsy`JZEuF&Tr7PzHLt*P{jASW1vXE8wMy5yfaF2L3{|XTd8<`hOQ7sfNA<{5k8_!YBj_7H`zuvV;uW7hTrtL_|DYLr2KQc#5WKD=BoA-g zFUPrd_}TR|9w&o8?OW(yB~dn~3p+=Hh~_l$graf_`UqbS;y>9%st6lM|RTj6uJGEp(Lhp}dmhT`Jz+OIkLg+f=*N8J$NAG`k?&t0&Rz^nd zPb|DWycx3q)@fAre`I=m-_s1n?e3(zt=;1Y`w+@PW+T39t&}UUZVfxE>mB{zw;Px4 zy;WQr6QfL#VYgzpi;5??@05ncs5S{rsao)2gM8KG&c0K~g^=9Z@yHYTqS)=T9Xs zvQi3j^>TD0jxz0zmngwhHUj{aoV@J-bSKE2Z)4UL8$>)8-%+zil)oOgez9fGUzBh! zB%(jAiHaXO9qZMibWBFJ4rd7?Gf6KY&T%)JR9RI9)RU>;2BEiy8mbh?YGxVJU&nl= zU}A!j$0kk9*4EV4?pI8_nY*}oj%m_?WQW6OC3JU?C0Xq1vgktW6;e<2clxCDd|zJ^ z#0FFM8IhF*3Wx>pY>t=Q34QyCe4QVh?R3Xit_kWO#N?1PSJk6+jBf}dA} zRCKBqRnN8Kg?>HHebxtOauS0t>aYxYoeUMsbpa19Tc0Z)Dr*FelymGA%mF$n#|wwN z=Y)g$@MQJ%^hVR0va^rXnHA+m*ds+*d_C+u-Rf4|UAumr?|g~7z_U@I^^IO!I%E!3 zGC?dBJW8|fCMcdH9X&d^U=MD)4>nz^V%q_(8n&o-)kD6-5^j$93@_nE?d~jm1#gG) z3?J~+8#$P})Wy~PLInnuDs4+!t)<2`43iKPp*L3-%q0nl&dpQD4 zVr)getD`+z5c($5Cs+&HJ$7viXN9o?zdj0fC+%UVAvxq78gZ=aIG1qh@c5g3@9%@6 zMk$G;^VXa-e&3)Hrl=>6~IE<&={1?0W&q7cV3WYMmHlZ?7wYpO$4TK0Us<66 zD!x|x=1%QiBAA$W_}eit8GG$AF-~5(S{+V-abENe)~KXw1K=B<%wv&uz+(qMwDjPz zfWOB5r8Xu}Gx|QiF0hk2SmCY^(#&GUOkV36XK9|xrqcMr1SSV$v*Y{y*0Wx6AAP+| zYQRKvJ{`IS1)7P6bL8uygFE_r=C;2Wlw-rkr-}^s<|WVDf>FdlmDCXkfHi^`{mIt- z{FdgAav&!}m4{g34Hx|2u`q!gKY=Vik;FDBCq(=$e(Y2bUq*<4Z1xoI1@_?%CX^S# zxr{KN-0kY2YxQ$c~tyfThY__XGrb&DzM|)YA(yI%w+a+YdZ#( z@9T%a?XiH8r~6tSsfJ5y=+MY_7{ptNMXx`c!s@fSz99Rd1+-zb+gYoQ-13DCSr0(swB40%H{K6nKqsta;wwM zB0BVw5CYXJE6ltzcbHf~h9aXS@;qy2AJgf!(k8&Uo{f?zQm3=AILmZECaOgR_A&T% zLUzUH&$s64jjze4Ekn_{L{5SIZY+|niknyt-37dQ&~;kVCA+YOoBg)h*yUCMcf#%R z=cv7gw^s(W_ciZ ziELhk>M-5fV}wPqe>P^Ddf8nqo-VGmwtJh(##=-2wlrO#6Yg&Q zXQFv(Dt~n=+|(;lEmvhdpk1FN^ZZ{dvov^Z*ez-0uNQx?jM^68nb(v)MP^@Mz<=TY zU)Go^*?s~87yuv<@*f59|FFh(HueVpM?3r<_~I7Tnb;);gx)hs(H*HyD#S~oU@A}b zf)HRNpnze(8Y1!x>z=I5ifij7^dnGr#!*+_EL-&W^MZ!H_tUYXIR`p|6ijqC>T>Cy zLm7swnLsy3Ev6x4U3(}0ZlsC>Mf35JK*_soBQQwYbz*V?fhIAOyLWvy#eSyBfC!k{ zLQbea)G7!S>UO&NQ25I=xrT(1Rg;O7Z9$x~lnRtl#ycCzlm^IhqG(EXD%Eq)df}-; zCJZl81QuyJ=ZFf6vX6Wp=_ZBoznPoydpHOnapf$uB&S4a^{sFV2!;KBU;^-~PH3yw z5a`hFJbR)1ed&Ze++FZVX?*Tn-e~p`c)Ue`?EwXE)cr$71Lkt4p&rZDleX#Y4rsOC z0WzMM+gA)$lMVc!Dx82C$Foic;;UvKSBAy(Z3aCh>xj)P(jz0dig#Ch=T+=%LW)!C zzKx8<(?xAi+)f9Kg)LulFs{vUj`ZUxN8qHei*lZ7u~*y|^$*40|N3H|5E9Gz*V=V5 z277Ko)JoWgMxY2GktT$~u9YPHu3-MYl9+POo~Yh?x5jdgTQCxLFc>hoq)`#SrCU>T z3CMclF?zer-eH0IRn!ZDZ(B=w%2o3qbz}_fbPw_P+0)EccJ6Nm=QRIHZuQjGI8yfc zM4w#c60a!Y%Z0ZniPg=*Wl|{OS0wzzk#gj3v^WcLmjIYGX7L$uo_MqUlIL(9>QOy+ zK>RATPvBdw4nXrC)YUyVOY|hqvA#wC#EiP-jB3%|f@$Je`mxXVdS%$9n}$YO+fr_h zadYV20oZ|_p}*tlpoeT<-5la8h;Q~UqDTQ_4$*Glzeq2{NCP2m3)>|6 zb{(!`9dazIA49j3u-VL=lssbJ=p);~<`wtDv0lzAiL&4mP?=!Pa*JneH`cqd`VJ4p zC_WW1K`AT>a`LN@R@$#Kc7l{Idn~K?YCFyUNy78KbO5@-cM16rxWoQWef`Tcp!87` zy!20fEg}E_ApX0)Of75;tSvnMBgy;^u7PW{ov1BQ1iv*sMs0jcbgg32c_NN;ohbnV z3&I}WltV2kVk@UAorbQ+qu{UGobPzjjfb6%wJ^a#axa#=zV{iJ53tldT0%9@v@DpP z?jDVwW0HxH5s?Q`)EfSJ)FyfT2~sS5`yo=~FJjAaP-ZZeOi_uUwPH;wnP2v8_2pxP zbQ*E!jAA~C4NS5LdaR5QpyXh7>7h92w}X*a`a}|=nwsR<{Y(q3f__i%zD8Kd(HOak z<;u{NN^6@chZZ1Rc1bOB(lR7_QVvklK^p&>7^IpJm3i^Nk{XJ7>K112PtB9pdy^)w zce)vs9r=!X){ZKfM3RC6$$Q+cOQIC;3n{d-eiX@NI!MM9UfB;Tl#D ze-*E8{iwgYT?^`kq8Z4!Il#coxrUN19xVZrsaui7e~=Al9zH!X-SOK{MPLHj<5XIK zT2Unjv!m0SO;NI+8~9U}IUrIz%y>m-U3Q^LKiA1z9PWZo)-AQu12XNXppo{HRz}K0*);O zph^{MDEO(Q4C(>VF7bq9CbzLrOkms^mjX9JQmva{58$@p@&nb-`rk%mnSu^^UPFuA}Jk2SHU+sCt1nZO;JHqxEK^rXX)% zVSfIzB1Ur{n3GP>ppnTyD?JO|kl6C*$ zLXb&c`d?T%#wEQV{ES5PLAKLv0*_l%uXu5T-6AlRv4TZ=S7d+IoIW<%0?Y}{(azb7 zJWfJ~)gT0#8Y}AwbR`J#Ag47e%uF*Lv1>xXBTio1?-UsV(FjN;d3gwNq08wDPq^8T zSc473#czC0?W;&Em1uQqxq}B;=wrRE4BZAN?>YuWDWph7?qCROlG=_S7c|)Zy@|I3 zE+ZCmxQK0mWr>u!stw+sc|IdN*G(4lD-R0!V*laS3FKasDe7kB)7E*|@N6%9XU z^mJ90I{Cy?h6E#%-fuD=8?<5GQlMfWi$6*KL*=_rc6;UH^|yBf_s@gdi%-_@&l9TG zaUiZ&=b*HHLT)IJ{fR~9?|<{|+%}|ns=xvO0Eq$sVElV^urzQr&@-~Lwl**@PwBM)bb!17sU4vUPClTMJd_zr(4rZClsb zCQB+r>U76UQYPG?w@ozQhfkWvpbYhEUztOh?J4Bhcqo|p{(Z&Z`n0aa4#H3Q>5z!W1)wA)Csb}b5HAFY2Fl2B(#IE zXk=@Oan0JlPYrw%Xi~aw#08;uc7U9vhiGqS+7E@}17Hm}bx)n4cdt-e#5RB z$2@GONPYgX1DM!=A#1pTZ}R&I59H@JL{mYItp|XR{`!DslA|*FWCU^1aZecY+9F+R za03z3MIHM1+VA-K1tb!#z|?@9&BD~x%_^KGe}bv1F=PXIVWO-ng88R?^ntF2Y)^?u z1ylz?R0T2z3?=G>YqLb~f$>WECj-*smfFJ$`6G>2jX>uXiWxZ+K6x@p8pLOw?v*x+ z`&ipdLl`Hsfxun5;2~ukpd|pOX}k)?#$T#fI-B*0cP=~zi_A| z7(B&mB4T5bQGKu|sxwL2lK@O70}0`{e&bH?Iegcw>|9tGtasyb<8qH>_GR4XT!h)9xO)w-enyzW0DaR9|N5DI!O z&sz~X5y`SPkLV=S`QYNBw^d;!10@$ovQ&x-O}&?b>JmjHcSDrM=y7KQQb20+Ket7( zjla{D)`Xr}f!p~Dk1Z)A_Ul%(03^>8ZzcsSMWzQrj5-C#R|N`xeq#dz%Ln94_6R?^ z6_j9iP6`0uxVi?*8SZrZN|^fZ&~;Tey<%Jw#G)96=@8j_>JUbVvkQvpjs7ABj4`23 zfVD(WqDTGJTxRJ-F8}BU{3C?$0^9#4G^DC_$C09yJ{|ufR(+L9eIrjC2^Fb+x&bPUHE(%vVSoybxsasJ|7Ai*`YCY7))oTGFazTEg3%fB7!Lr)D*RVzhKWW zsDo@C`#tSzOav%TG+AF|0^r61Bv2++6o3khwjr6= zu+t_A$(zGg2-56U)f~@v*PW2*Y58RN&dO9yhaRTSzq1jJTW&5t5@SI_&b>EYMwmZY zk(Grx;IVvcJ166XQf;x>w_O99z11AhRxaRLRck!9K2S^G!Md3Md!Vwtg_Mg}54w*Pr3p3}y#nCS12~LxFY= z4I7qvXcO7^@OpI^d}5! zw4!L#`vEczEn6u>JyACk(aGev`CJl|A)4oax*|C+Kwm_@3xq&+HCm2hO6&BVi0;se zqCLNS5>)M!shC~kCjw?pseaUxk|1Sm_2XmyoYsFNah7bcqTPiVoA5^%Og2iBza+o(ceM%ENcj z)Kn2);Ssx;nb=vM$eEZK@XME+oK@P5oVft!Zv>}2q8w(r`cvr#r;>k zYar73#j=a9Z8Bc}yl6;N(NsdoaYZ*`2Wv2Xnh&Ku1TD?{?^=I_r#CW1hRsNJ-me%_ z1i2{}$kh=11&1a35xksSEO?xeF9;N>YkD3P&$s%{F~Q*%SErd!4saA#YP|poj=!dg z+VRvRScT09M9{~FntlS6wJ>N4#nfFsE8ds>A@LJqX{G4j0|%X2aQNL$3!!{OYj`aG zWiVsY76AL99E4C5wRp!@^R_^ctyHoW8T=Ul&ixl$kvrHqib9HNQ$olJ^n-!QHqmhu z9+w}Gu2yD+2f7m-ed{m!dSPD-BZwpnze=h{WpEq;N*Zf*8g)L9x`GmbAmel5HX1C4(7tciiCf} z>&UdHzv82O;fEjjt?j7VU$NOx^p+@Z7Y{%%9m-}filb^48B{`|LLJI&8OZsd9r_W0 zO(V*fYfK$uwB`D7%QgPixX0p;mKMt}{0XD@ZVS`}@BoHU$Tu6>U74&jY!mxw!H1)Z zZtVP=xY>N11Ux+Z_EgJCcS?_t+EO`}x>N6M4zC>H#5b1`I9{M!`Rym`#W%zZtqou* zY4pR-w!iyhuY_F@2FZ{*V$9v|y0tO}QiaSea<3n+%qkpkE%_L?F{+7!@MIW4E&A5k zVcIbhCK}ju|0)Rc4)Axy}V+TtX)i;s(m4nwy$rqAxm5FA|4DXKQqtHMVXm;H$wgtbtM`w}w4es~`EAXwq_1mGEWPABXYRpB33mwlG#kd#t+< zdG-FnBF7>lx_s@`yOb-X82q^8`jIS^D8$?Miyq`e2SgQEAf{pvfiTk-c6t5zJP zu-Y$E1u46oUs^b)YeaN;RAQ3?a8oUfzr8N$sn6zWhdRJL#Q`r`C+yeQSt0BymRf0D zZySZjE5_K9y4TG!CG@_n)PDDmcSz~&_ysVda!mmiWP4KwqmFMsVlQRnLknmnj0qdS zpSNevdvjO#XWc(mG?6p#!9I{f?8``R8+2V`a5)D_s;-Nnk9=|+>tj|#)Q zM&b2!8rk?9$|5&I%Ltwc2frt$KFA#0PPUASx;2yA8T}_!=B{9NlGb%uQaK7R4)S3| zc_kD32`NNqnoKM%h8}5Z)-h=@q(X7AXR2Z=HlXd8>oU&^%9^w0NVb!nW#U|rcCs|S z3s$UC&}hH9Y-;W1@O^lwL4SGUshhfAOqY4wY7erebY|Y%-Ob2D?UGIux%F*p0GVS#1Zt;^8KweJAqgtvpq*K6X5<)C(?Jql zE7Lr*aqqXCIeKqh>jVB|vRh-}qt;E?D|zK_b!RlCh%|cKwT|zw8$()rSngJ}VxvtP z#+<+RG{xtm=bo>P>Nv{u5HG3m0o3Ry=%YF;{o+r?l36*?-qjf~=BFLwNqrFgh|NT(*6uKlx_c%~+G3 z3%&*~wd;2M(xyBIyo|l$<`0W4D;K;s6P7k!%&zvXm-FWt9b3hn-+xG)e_3mJYA4Ce z! zD$#2jTmYU@p|5=n$e{|JMjUczXuAtDjoDLq*mf5arEniAPA66Gb^NCNmA$Wn5_ zsnS=iEafNB_>NP|rq`6%b#i6F#)#{w8iWY4ue=5JEVX}+fa=zO&FDcPkR7=m(M5b8OY zfK^A&nI1HqbkrG>0q0@o%ppwfP4e1ibRfJ;dChGTA+!io%&W zf!Sq=$4REv393fQ01br3gIQdmkDAeoFGD$~JiFXyB?oZ!|5?!4M9&peF-;>D{%F}|=u)yiOsV&tPrVmH_*vI@v1 zg3x!56aSFhigdY&HmJU96J5=)VddU|9GtOc4a&JN^bG}^l`$8s5QefN6#jt*_Pdpv z;GdjKQ(n*;!DQDbSiZo%@YDMI+vCn1ixyq&KJSgbNHFr>l`!48*WZ`5Rfpe`h35+f zPEOw5OL(xm_V1?+zu?ciy`@$kFAoRSV6{2lx1Z13ouRz2I{Mk7sj8?v-xn=zH({@y zpA!bG-0Wxi+Jd1Ag}UF5!>4w?kfU=|+vMGai;JkD-MEE(zh|_!YPT>tzGgZ<-ybrp zFLy`GXF6HA&z^&ckw1(*&mvh{@O8BcyNW6UpD&sz*R|$#_j!>&l&Y2AfXz!(Du79G zp{b4Zk&{$YPWEA8o3{G7{TugY`Dufbx)d-Fn-}0b86-x3IxfNi5s_mt*30Lg`WJem zkn`&&AB|;V{a74n#fS-`-l4QTRzz-b%l2oMB9Jzu(*ZqOSrM>-0QXHJt$IluOEXRf z`+lum)TL1&%bYxlfQ=4!YdXfS_SpxiH4-FZ4t>UUC581j{0G8#pQ6dT=;Qs zk^6ov3Lol{KCaG|Wscb}-P6OS36nmd1bSUjJ4;^AAQFMemGgqDkQ zz>h-~x2;V708BJCezX=6qOxr9DU_p)ItkK%gh7s)17OTVS*M7xyD_+9o+H5Ld{{vC zs|k7G0LuF@p`vM}8Wt9?k|6-Bp)*u_NU;d07PD#(hxeldddY{1(w|s1@|&}9c(^$7 z9xDTT0w1RwJNF|we;{DRVxZ3eP={53r2nFaCt9qy<_DYjPkgk%9#zO$BF!wm0J{3i zXUs)?F}J8lRJ6VjMf8izApM~)C<>Y|xUKYK`73tz?G_I2=}%LrriQ+jmZu>;>YbUo z>L#IXuTlvC0AJn+6Rm;h?p6ubtp#OC=`59Sg5z$Q3^7pF#R2+B*&<8-H;Asn!X8oQ zxenY1gs1dYabnrnw5mAr5yU36ItbGrGnq00n;LK`?vaK>t$Sly`sMhi`61Q z*9TB<5~w*$0aKrj6itE+Ebo!QSb_%+M(_&9(#O;BfZG+61TIB_v=gWX%LnGzAj}4V zD&Qms2W#DT+t1!jA24~fkIf-2zbVD!j~K<*)sNW|do1Z4yHY*> zU6FoAh^A@V-%#nx$E&5eYro}Cc)(#AlNOQGS+uk})T7WsM#oRA)aSZ6UT9jKH~m>l zO;ff%0#FBP{8+a{Rk%fBRr=(RJV;}jKm;GMXq3_l{NL`Clof~8jBoB;(Y8h|YbzJT z{3(O9$;h6gfaKs>DNfyoBl!vUn$g;?h?WqZ2#ERg@xA;GQuzhVVqD`3 zhas51{QBX{d+|W}is2aQBZu9p5A|{^_}x_S+x5}`8qmrd@LeS4?Ujcl+#$lO_l#}?if zH1xi@yfAaJ4d}37YxP(*IfNX6N0rXEe#ACI^$mIS2(2lt3V#Ze!8mJHP_Ju}l*DDP6Hax(>OAHlcp5mMRjKRJnrQ(0%@)9j7c0 z4pI#ei7h9}19J5sRQ#8qM)VrQL2w-`$T4fKI;F;5!ohd_^pipuP2=GU#Qu>5PzBvr z*`U{`>lx;xAmzy|UAI0B`LNiN0WJlyO1b(xL6qce*a8Wn_J-m#vN!3M+WSXN4l6tW z_sIxNM1(OzulZS^hRDD87$aJLKy1ch?Jx)`diw`Q<3rOv)$4Ow8-OOT*5R0x*DTGV zCVLauCPQdY8tP>bTM%)_l2jD@0dG;M6dIMEbM^FFw1?URXAb51`@f6$kUX-P+rtQi zQ@h)V4lo*RE7NtL&?X$g3LN1Html6@gNLs{xr5-(qs${!$-2 zkKM>6y2Dr-q&_YsQMS+VT(e1;uGUS5yl-L)E*MtDEEZow)=YVipgFw05a6NpDO~#T zK=#E{;yzBUJGmGZLO6(ZfkIRF@enlQRS`_0Q?5tUs>5jlS-) zLCYLd<@M43+p6H(HNS351NF`T>lUAB0R#>}2Moz%iRVjh6`(X29A(tq%wKyMY!06;3J0DWe!gm5~4?Wt*V;Huf0Ne_Um26YPJRTkMY92ywM zKjp^F44rbke`KnU)u|dO$I3C<;@FP?hR6`VKQf$x#DZxf(eFbMzrk`;>TvO32faj1 zrBfn!0^Kq^BfW(72p1CpLvyy`v45AGYqCu-)K;KsRyl8M(jaP9@STtQA)7jyVMG^# z!8^0aQ>`px!f3)0>VYE7?YzdydI;Vo35y9S<;$joVK*`kI)}K*NwEZQkBZJ4-m_Vs z3Uy7e7TjzbHqxvof%|ISRY3Wo3SO?Efx>~w3@EB9u9Hq^ANgtamURx&a+I$t9;VztpKu; zWTZ(nlXzN#!p=z5S(*|m#LHmM?L;?&i`I|UygZK`#jmE!DNEl?chq>`Lck`oLVH6I zdI&U?;FkXR5E+_c^QoksE9j^^>=A<9r1)j1Q_?(}IN9v}2GBtOE{GOof>ahM?Im>W z>;?t9dsG!7YWGDBWrKs4)lXNzZbY7eH6o|0S`VbUZ~G(=w79BdE!QS$(*atenHC>0 zj>O|N1X2+k?HZ0sQOl_Wxop;15vmAMy3YH$F zfqIE-Wu+IPB_ib1&&%oQg~HRvVbzZ*Rw=NNL+++Q$VV9)R4F~MMjbkeS}ZC-ZYS*q zKwvnI8LZi-joj=$PA}>11zEjpaxcAp3lc9F5bh2;YYdE1n3{v!-|3d%oTEQb3-*w- z8thqu?orsqg77V%ObvBH_cmg;v3+XsmWo$NrTSrVfg|IBQ)q^JAjYv_H}O#bj~e+2 zKs0fRHl{3SU~?=-H+VLMfj>@kBrfFXR%{P3X>eK5phZS9;(cn;%UUjSW;6g z$JAmKj}vN>2kO;0OTDwvg^J-(@0D+>GofIaQ96fHT`_ZquO7qhF*2ZXgjrRjWKu&a zv^>r0A~L8-wYSX@5cg)o)wlO0;6Y*(@2)}j9PN+?|8Svn3>)_vUetn@)4?|(tPM1+ zhsV4XJl{dvn4M~8>PWlADm$~(L`B|lqe5l2%(6QZoby$X!h`@`z>T=N&Q@u)`EoU3 zJvc=D6XW5N)7-yxoP2B0o2N8?N+|EIO=N~``yCMalE9K%ZUx&YKVPLAy?%TWk($}Vc;wp+6B(R>0rYxcCvMl*dZJc<{z0^|M`q5=bmAqY*z&q78miz z2j#f&x{8m`P*#p=tM(23v`W~49KdN%lsSpeDsQNnAaI+8D}g)G%A~h>Ra5r4Hz)Bs%u)l=T$qjMf4 z2Irx???)idJ|`>m=q=@qBKT$yEdCHn5e=a6d8c;FBI1MA_EZa`#>tdJJ?A7{L60J= zle&uAm8yyul892L0t&+|w3v#KYd#YC7_{O%T2mbPr&&ySM%PMatAx)1jpVAzn47~H z1^`{sGeIU}Z%Dw8$u@RB-FS@L>=__k%`(Y94dM56{BuV-d!ex&3rcAx%U{EK&F#pS zC;ijo?VFMO>Q$fiB6n^@IvK)3=lvaTS88fR+mU1ngNwFQP2IwC4N*BaOM!1v%{!Z4+}pSg9$@_8x*lJK69Jmp8i21%c%z5bNmt*@>|v*R=C0}M^8J>PJGDu!*9ROY{*Aoa z^)6)i&|`UPox!JL`Mj4*7|ni3A4H4Axt|zV9Ss?5#Y_!XThm|LQ4z~kS^1Mu)B1*j z+x1k$9RK)uy9}wi1#V1V{0xWdTwUXF!?--rVqVOGxG{2BF5wjcE;+C}89@W=X-8N0hc}pX0n^dx& zWJRIMJWHpv11g*BZ*xw*0nuk{t__n_=BJGKFWyUb+WI@bH92cZXGgh-@fleDqYEgD zBD+J4;J&zbW69Hp>3Se8TwgrKF(T@tkqIBo1!%|IC{8^R$9&^iR+U=jw3ETMjG@F#K4ij>ouB>-80i5@aIfWgNd-A zzHspRdKg_Dzq|UnsiWISsLz_Zcg-Xwe&`_p70B)PRt6W*>B;%4{uRMC+S%LZ1P4)< zxEc)UfHer9VOv&A;;~zcW=G7sZYlM_T|8l5DPs`&IK&LMU{D%7OT>qB7IxzVE10ZV z1UpiL9=am#E;1Pd1E{ArYrXS8m_h<|bNvJ9FXej7dLxyYL^G~j*^;3ip5`&)411Tp zNGU>ji12mOOQI~zW-|zCPxW`!H9Xkol;ysAGWXt*;W=@DjQ>PHh*J@azl-DVx@PpQ z9kyd;eX>xK8}E{-o%=3nb@K*I4?E~BS-XPitb&}bc@icnzEyZN!2EduDzn7~((6rA zo-`m|3M%7PI&?`f&j73mpgbO(tBNYa7R@x7mF6?n5=KfbGof-PfFp5uNW7 z=w8OWP=>~t52v03eP+)+;eKV6>7(eDTHP%CSvPl=_QU=C{ii0i!|+)uYbG^6;=Q$m zK>fGDvzp`3Zw#*+u8!||8RYOOrxPcC@xY6Y)b!UG5^wx%+}Ete*5e$fhhp4rH=&#L zTtR!t2S?0m+BRuz3!Zegfk)|rWu|AJpyh!zFPjDYaMLEu{Q=Zef%oDk0$7pT- zn4h9;;!yP0I`lPGeK&=mW4g7{Tjz@{1)gE5p|6V6t3$CUK;)c2t5#Xaj70{4&ehbZWuXAI{eB zkp5tX_xlA-1RjE+G8`q4@KlG%A)hiN7VsF4tiU5dG?`0`j>HE3r5HTAY7%pZE*UqZ z3L(?+Ghvg}clpQ)e>ul<)Q{T@IAIOKJFP-xM_RKnN?c2@?eRN3Dxlz!B8S!iKd|SV z*lzHQ5Kz}AmMdLp(2UpfKr_xRIyLU~E+HV zQ6z_JcJbm(zjtpn5`Tb$us(jU?Wz?1Y*bsd!FImb(An2U+k5su$hH3FdsviZHx&#i z__tDP@APOec#u|M(EoRu)O9&kv^%uk8?TWuU(rCo^J<%$%=oiWAT~YRX<#}bO1aG~ z-(r(q2=Gpen4X6;0j7)&?#{Nr{`}TT(#TWl(X>7r_k`A3RCOQgvDJiC{rk7qaJ2X- z>O&VAgdZVAEkp{dAfkc-MgQI2w>GzNBnf`^ujqj`!hjcor3vrF26yOPnxbWnMN&&r z_Bb>Opb0d|wm<+z1EP68|M#nWRb{;zASKzJ-9gM)BvF<1%F4=jBC&dp@I-T>U75bt zRDrx?2WTJ>BE=~MQn^y_V2_DU?M%%fP$5WHkDQoZLiW#1!E03 z4S=~|TX#XXtDwD`vHf#oc}gxl==e~vm!aFdY@|@2G2&=^`XnXd=!leTUpl% zM}{K-JH@1$8qNXG7zVMTuM?rbsmH$YqRE5U0(9|4ebNFSZ_ry%c*@(^{FVeAj4hG; zeMbedE?^d==Q-RH7QI=yW3xomH#gO!E|J-0x`pY0#^$2aV{RSK`)vFur?GRC439T57B?| zyrG6*)x*?g=8r@0?i~JP1H{@Ohwh5oK&FI@3bK8_9o!D;nL77`d^u5P=vKkUh{0<+ z>AfD<5V#Nm&UPCJVVGP9=yI|R6cwxI04&!t?5lcV9N-YMVXiO{8T{B)=wUqq`yk{y z6a!fmBTS4a;RD&U*In=N!6~{+7&_8zzJ(G{AkK(U+#kjagaIBKFDrTQiQ0WQt((>6 z`x?yto)?x~O5q=k*4`Za&3#mSFHX5*SS4JqmDrmb3K42-2&oxa5kizi186^jEzQW} ztl_#D#7F7P#?03H-8PTC@{^rkziCBIyDI}oX4lQobK5A4@X5A9k3@~g&bBj4Ow?aU z(o|*fmtQ!S0H{0HmQaIn-!I{YOWGBANa(;YEr1ISrZ36X3>rJs=FlrUpIz2h;L%VP zBW39xwcvGh9DPX=wHsvJ?4zgfuFkgEFO**f=?gq~_$_VlJgpHr^V>(?8QDUXN2;e*Lg%r`s;bKvbxI= z#q^SAJXCMcfpiU<&rZ9Bxm(qj6xO^V?9eN(r?P+|^lu zScP%F^ZDELg4~80jMlxgS~vn(qbV6wgctr63FB(LlzVv85_!uYEikhNb5!*h4Hhu0 z3WkKwdI2{J$Ga=8YqX<~CbJPGp>kzH+tpjwCYo7OOfh)((i}9(VCh}OTNyBkP0GHn)~IOvj)d3|jW<-%!$rAFH|3-XG6&$4U^g9QjW;v%Wau#F z3a*Zgns4e!#d$!zk(A?0*K=;PiU+hbYcM@P5WWk+Q^EH_y0B0Wo#M^%t&nEkUlva= zZn)uCZVZtEPGyX4IOXL^Y5w5j=(JehS95bU{FK1ferF>l>_P^~w-Y%dd6;(Pi3+$i z3M;bVN3NeG@X*!%l4wM*ofR3j!3yB=Zj8@h{Snz>&BA=qIi2!Zam09FRC0CMK5 zP%=}7I0Vm!%o3VfN=qVk!1Xa6@g9L zgJ<^a3TB9}V;bBNoZn}4bDd(aG#hak@Or_LK)%o$>R%oXDYb6zo+Gto7Vq=M0~GgW zf7*EB2|xIssEt5@gtIKk)YUO0m`m?UZ~bNl;f#VuGNz5Ojo(f;8yLi}Ka^DfyZUdkq#fdoP%`rTr94QWPl7Z{rD2|pP-J%4 z0`Ster|{-?<5XZCn5mXc-NLX${MyH2UhHKw-`bxuO*CQA&4(#M@*(*Ypo-o-0LQG& zuUn{56eE?3BiAXEZP*T+$zU)DJD)r|DNfN^=?bb8U~Gc)0c6Ichav_UfWiS<9`0iI zKIya_3WPv{{MlaPZJ4%a>zkW9tghsRBg=5&U<=j(Dg*{}iqLJZ*UDumizmxVddLh!fF zi*Pq$i&|IBCVTj+-pEp{DcALEwfp#Kx`!TE+eNz2tGDH5F3WL+#U(tK646Q|oJsVC z42D9xr^m|R<4oM<5uN)R3@4j9QCrz9?-)&s~g3OYceHN6v!fnymw)3+F~ zOBWIh+xWOCFG<&;VF9}%nyLBh@ub51Fbr=QbOm0< z4eVfDav&$=I<&mh;NzGj+KmulICMf9jI9o~lNF-35cA?rJ1qGitFHudamsb#cgV?j zCml0g;i7gd$hTFsP+1^FY1P1P~m6A(%1rq=49F_WJZCE|X;9(iL$yjRTGBY05!d#NeqDELU z9IR?*ypdybWS?87DSzTt1lND7dpU^qyB8ykj!^f$2v`A^V^d*LNJ^3jB~;Q?I}PHZ zaOplO^<&MRSi+Wa?Uv!0t0Qmx(!$iX@#Hvg*D3pmjc;R6m zWt=I2)+!w4O@7 zcjF@NUYchMecHL=fE4gfDU<>GLN^GHvv>M-mpnO46uAn#s6sshBoJd+ew2JwY!wgO zVwJbrA`M4taS-`L`5Sz9RCHn4+=B2#O(w#a9&OXrnsR?96dn}6(9CV{qK6eDI(%$J zhyM}ir3&HJq%#|vrsl`E8NHOYkQzbm=Qx`o?o9Hjh7V`$(Y5Kj0<#y78tDp&!UZ5h zJtKU0%Ti283mXvgj!LsU|A5UodT6h_`0ahD*nL09-wbmKZvJK?1hHy39;GD z!q(9V)6S4BaERoti~sF=cY85Qore_Yg!e3w8iexQ+f3T+j~}l`5H%LE>`TY58~wA* z;mb6wKUi8|+q{6*8ME+udU|r|=udD@+L(F1DlI%Q1IyStqK;}X?|3EnCw0i7W>Ly- z%}5;%2#U(oU2na^lVc-qA3t*=IwyCZ|KP|q;8mR#n_|*37In%Hj43x+M~@j&$Xmne z=_QMBX`U+c*j1O8pb|0!-|W?uV>?`)orqCZL+(+%Cr|MYhf^AENO&RUr)gA_O+9xH zC$=`?(+d`AE zoq#JJFVG-I(A$M!__W*YA~TP;0a^D@#W_esUujOF7xNpol$LVZwAy8qXuhK&dc{!* z>HDieTL4Zy(#L%7dxt-Mj_{7G*y-6mD9ZI{pu+>@6>c!1@7V(^G2r zC4h%-l^(k9z(R6W^S?k%OGswn&_7*|!MS+i_ftXYOHFngOQLr>2r+9F!GBwI{4GQq-d5#Xb0r|{V!{yU z%l_rf#iT55m;Ep{cvtO)zu_&>-i0TowzrL9hP93Wm2wHS6a2jig|7=oP-j;*Hw&ND z$i|P4;C76XS3j+&V;pAK-Ry|Tlw>yxyQQVOYb0s(3$iJkY@Ju-)vEPrsa0V;6ScuD%`<*@nVk_c2YoDhp_ zra*PXZSEwGiZk>UupWaK)g{D@a*jngKu)vB2!!RYXki*w7{{NQK@`nNAln)Q5b_l8 zx6&u4;_x1*Pj*UV4eTNMWTiybpj!3G_q7fMW+9n@I|)%Jx?R1Yq;LW&qI(x9zQ4MiOmQ_kNO(&h72ji~S0rO7ykBw=K28r?OBz?oC;`@W^O)U%19T%5cN{QM z?4zfi_wIG=o45J?i+U-0rMF;Uc#~KWr|nRzSZG(iW!QhyScbZ~aI_xvEC z^t4l_A6`G*c@rSTH)x=4IEd4?$xtnalBz)%?u3XX_=nCY+iU=fM`K<_#U-dPS2&YF zc74lw(d~i4PNziOaPYFFc$=MD5I9xB90NrtD+y`D4}!IHlphIiE$}4fo*wKU9DaA; zypUBTi`WHOk`p?Y;`XoKy9oD%>jVR&iY<_DWG$ck6eUJXX%V>fPtnY78%` zB=c2JFw?6`fNe+?5+W{tB5otlO7r6JB3c3p{I%vYL}n17&raHs|1q!$dQlTO)c`_Y=0qw%p0y2%#r{lK0{B2#jc~-(tkDMG&$4 ziRK|C6 zE-Kx-IZY|itY`t#tUwaYjh$X;u*@(%$Q9BEM)AbF;Jp7SH0O5ueWYeM4@htADUO1G zEmQ~@re8RcKm+**(ktVZAV)^23IRGiD!`keYa911nt8z$!P1rz1I|`ZGjp@C`T(Ss zdU0soY>L(GyjZN43ov|#b9&m7gD$#oc~E!T{AUkS!9dm=tRGmU@U zwxR;?FQGNvGiO&{tXDMRC(*f)jmv*MB7H_Te4-NSgf2LE!*_j;dGX~>DhE$oQwRXY z5f~YDMf8vX!ODQcEl4m^{stOKVD2m@r0R!Gr!Gms?R?q5A(20qZnz@aCk+!=z4RJ)u6rNYH^baR?z~x$XiZ;*Bvd&yH!_EB;@6-dcC!G zcpMA^L-B3wknsA+n=nd`j0T)g)>Q?APppgo*w-cIon(UK?CG&`UdFMXtVDpEvo8{m zBSccLVfcX$8bkAn*gyugu@rOVlxLr~nUTlBzyoU?+t7CbLu8@eHmTrH0Hi|OP@HVJw295k z=M}lm^*wOx0HJy+t4)nYR4Q6Mr%JH`%+?GLagm%v{GhGTkMC|qUKZg}WbS@Z4Xy@O ztuqpLThn9kqnX}dTF$NpbhtrPv4jHk@~7D*FAFSvxeG1dORK0o8#XGzV%LUYJF(?J^pWdTeIeFRA2bU#?yuiq;G4S=K{uz$` zx?F})XGk_JHHca3ZM;y2|5M#ihgzZY^N)MgRLgISu)Mck&7p$xv`Xfh!GFOBZ|(^M@L$iKjMl7o1kg~q=3P{ z(a*Mr`de2`+VIx~tp|KoBo97iApyGC@&>zlsoYL*jOR#YArKYD7L!PaNg1f@`Y$Tc zpDz`@f7$Vr#=H|bwK0F~DUN}flG>Ca|9KRpLWaM~UFMAmOK0RXz;?htv<~<|%3pK6 z-rWrANtjsb!)SsHsuB1$)g!mCIy0_)V2H@@@YyGsw)DeB^&!*j*J=7ZFi8340kXA` z;_?8K%7}FH*eB}4NbOsA#zmPb&Rq%`Ow1uyNGBx(j)YLc6WRtMw0&T;!K>qMk0C$B zCcS}7E3F#v5a&139;l-Tt+%WtFk|e0yZ8Jci7%Q8^77h+R9c^AYFk3kWd|^%!qUPX$sF+mU=W=&KPVotfWC%0w^AQjn_8 zt5@ljGjgv4>UH@I5#N4Szap?%cDZ;u*6k)=jGoTPB{xsKAJ&96{Rz`_`V(zQ zn#kYJtS1974fd`*B&(Qc*)2uxk$kI&1;kW0$E~_sLUC|{D%?b&it0Vp&W+5AI;&|YO3cC3Y2&#tRwO|dZn9H&|# zo7Q1*i(RnnyG@-4=n8W79P)`=PODTQ!AnedeYKt%whR-X2p^cCi@fo>YX5>VJNURt z>G>gukYr5TExLqbuix}GzQy-7j=5H_hldB>Z|BC7rSg&R-L|yXPu}!?2WOuTC=0h6 z(=Z>68Y_G241*0K-Io4qe@wULJoYBE$FN*;`hEHF>CPL22yGl>#@d)ctfz0R{qc8U zI<-pU9;ljO)RJ7?!rr_+k03tQo=zO)|&-ReTqlB>V)+FIjAjm@jq} zbNXPv*mU2CxIsD0jnb_1Rqtcb1|uYALg^@gJK%e~sK;DXBRet0?n$|!Oyk^Fo{EFF zEvMoTUXx0O8tMjRvay0?Ng?fvx%tiTovYglb3ie-^4*bUDN7Zp`dJZy7Nrz0u{tkd zX>$vxMid&$kX7kv296mC%p#ZI=vzY_4cc1NVr(}q!(OS}1_Olnk^lx%$xTA5@`i+D z_nnN)WbRGojA3m%576e=as5#VMbHrpEhj9FFrU_E(SfJ2Nq|U+4cPUFBXTLIja63| z{6;s?pA$JrnjfK1E2VyN$KoArpidlGb<|{jZiediMu@kbmkU9$EfqM`%X-Q{QnO__ zsUFWSFD;Uz`lWVhXoKh!3L7>Y)al~m~*;A9UN&*r%)vmPl;GBSy=WX4nvuvPXQ-?D7i3K3u zhwNeQh@N5z9BH#63g8#zto_ZwFCd%Xl8V%^dS6N&$W0T}@ZZN6Gum_YZ&{Hx4?9A$ z1Ow%6wdNhXi>H5yoo3X1bU%hQY4f-Ut_s;N!>wpR=|D<4p6jX(x7AW1Tr)Q;9m0c^ z5$*}(By48mu1tKTe$0wiX!_wQTR z5c6vYZ;ydu16U%erfyfzXcVllCP=ot~sGcO69X`8H zJct*mth6q=qsk?SFx8_L5aV<)CU8SKyE8)N&48ny&g+}%sGd&iCN5;3DfdAypH#$2 zq+xdxL`z=cbtdgKMDpB&quR`hSKDr7eXt6`J)fj}j2GKDblDH@mUmxm4 zpkvRIE@uU}fO9inVdO6-Q{2Njm$%70US@!Y3gWFo^9JTEw719ns>f-E z7`x>>+6rbQ$0bfQzQXNA>x~RQt<1?8_I$ox8o-bcjbB!{$)ZfMDRL7UhzucU<>tK; zbreG#aFBN-vzwdlzxHW}X5!1UZLF#0`7(CKEzfng?Z|^I$kIaar-H``m@4&@>~_ZM zW;MTohpQ8mb;IogWxeT(NmG#_Lr?YEP(vuIM9#&HVYpm_YSyd`JLhmwvR1R>^-VcJ zPe{1=*ufelK^Z%($17K5RrzPb2P`}@N&$jy;SK|yV^}@!5fhtD4!OzutMT zXZ25z2!-l>KtjPu-K^@FSv~+D&ogmKA}0vX8w);Nv>DE5-ROmkPXqQU9lUyys%gI$ zc}5v#8-^h>X6Z5g$O+kck=sQti(k)XFCgH1_D>HCt2Pu*pw?cIuMe^~zojP+vZB_r zDstp$^WBMKPLlj(V7c1BZlX7X*I)Sss>$|zih!9>m>gg$Jo|p}*~D;oGqATcIii4< z1p6cIL)6R2ff%82$UzxJm&moHg!j70O_1WnMFQ~K;w==1=Ia+^dXcf*JOJ$>J0Ns$ zfAza-`=>Y3==Ib7pWlqwbI+jHWIe_NGb8K>zCY&cWmKy>GZf7modsmS{?lX~Mgr4%>fVCH8GMhHm@B%ly~fkLf_33uoSffav<3Qlb@3 zcxV}~RI%1G)1#r9h%0sLJmsJ*+^hFbTKXK2hsONPz;E*9D87)vM0Y^U83Fq?;zdL? z7Nk&_ccfVIu}Kzu?3$79(7SeeNBjg8K5H_MSB7H_4>PQ%x~=#N4w=E4BOM?ywfZmV z@Lyf_Ecz1H<%*`Iwk8m*GDeR~*))%Rk9g|@YG(NFd!y?AB3#T#wdpV`un=PvRa7SM zt|#Q*dkcrLjP^eq$1Uz+12wm|$s3tuz=#bKs)CcAaTdJP4yeF+EmQ3pRzUEcJf(Tvu z#oMY{WZe6FMl(e{hU%)c& z5rlk4M=n@Dq2)*|cLzWTK}q2O1)BJqXATz;2OQ>BRJGdR<8<9XZFs*x&NjO@2q(I+ zhc{~~UT+#FwQmW+)>f@2w8}9733bfLt@Os<+ARl%7^DE^eOP?Ip3GfKdE{(yFEJuw z^9jVT({HT0hpHS*U+Y`^dvjF2rKgq@61+;&*+pui*@6QHwGFw_KFp{A@QSo^4i2uu ztn=csDDFRM5%+OcL{hyeRR+EfHk7OJfIDbPN6N}(#h1LEy_>(4iI8&w zTC3}u8pm1EH>>%)FtX(xuhhNAIx#`UryV|8EkhtkFOyg}JiB^>yO+zV>J>qxVla?8 zvy>JsOvwTmfcZ)3{NI&E7+#bsP^Nid7Cgm=XZzvG2)S}hfHR5_3?sZ8|vS`X!Ii_mUK1UpH%`O-Rf3a~pq z=HIN#@?|}R1YA^I5@QGQ8mrD#wfg&KU(e?qgBy2B+i-Me{@90~)W2Vql-D96;TM?v z*|fm;SD7DR;Z3<3Uz>TAS0!Kv8GdG~8tPBiSF_TLSNZmz^Eq`Q$BdN}3c=tFxf)>G z?9eaA=F9HiKkLN18)b`?MK^9rpnm zroHzdL85RdJ2&x0&4EiODe)_{80hZs6@0a&H)eHHAw%!{W5@Pp2Wp5BmN*xHG_q=~ zT0)@1b@5mUb{dWut`MavPT%Oj1u4!joYrIO>#c9Lz0^z<{tGXwswV?vVp^_e<7=yu z3Al--Pman@;1OAxF|Kzzoj%=q`x_9V`C`@VzJB9BpOd;8&S!?juWi(ZgwG5c^ER$m ziI@xB!U$JPuO_9T-Lv}tRzvxo=@wE!nhN|FJm74_1a19dIS0mWc(t6b7bzz41@1%; z;UxwZ zN0x%_w=e?$Wd*kZ2YW^#i@*>QEyo8O1|X|*L{<$NP~k?jLz)@WATjpM7VtZry6a>Fy91+IbiJYAjUk!?6~4~KsRMrz_&Jm+QsK!8v4!!bfLA zx|@DLw=19V!VNb2VvuNRxb{8^iy)j`R1UF_+kq0DK7rv(DzHKG_0_duaohp1t3ml7 z_RW7Hb+!NEC!?^?4k??KJ!NzL!$ilT?nD;O21PV4uHofgI#N>zra}NySCAw~Eq`FGzbRz}*~0)%QEPCt`yNE5-`;`^#JyQYl-kBS#_gRxRw z3`*QJb6_&f`%_~&HpmGGln)aMDoTjYkmrRD`4Pd*cRs1m$ygj2tsT1|i^38$j41sY z>2m6&Jf>7|cUG@CF4w?O1BOkia!M}T1AmQESix>_3M&aMyLB@ZW0Ps(*KpJgKsp$D ztt=_0V1|U{+6C_fmqOcPlsbS7Ka@bjLShmCL<-W~L59ZltlJxCno(X3(HnJ_-5mxH zgk{hP?fM!G`wp9-lBfA!fNq3R1jT`)f4V$OU;cb{vj6Sy?ELg#??`VexD*;}8S1LM^W(xe`@oK$bKA~Q``#=5S&;DP&@H)U2$<6YIHG2p@fjT%D zE&}en6=aVXx;qZg^rog-GbzEQPhHc{=>v4$fyc&817%4%d+a(x?lMc8_j_e?E!)s8 z_vUm`^{qSNMe)m%Uw3|a`fJZG7L(X*4+wjY*!$Zx6HG+dMR7gQu3z~1IgZM^i%N1j zLg+$6u?q(OjWoVq6zj!*`nbw%cg<~rAG%a2lpNO074e`y;}Qbq+9mEdb7{neZ$V}) zzn%mm!A+JxfRn9%{0V9y(QsBxOddC^pG>$rScqny9Km~LRjQ~(9Ra8jkyX(UKWVj~ zq_8!&6n(+-tC8FBumv5FggOb<#Xv`>-?YDA4B!kzT=PTMOKy*v>FM>nS!G3rYe1d{Rc$DyxOY*c62<-%h4&%^J^*Nk2Ot4IDu^~36nh#}C_d>_bZm}? z$%&E#!pOmzftgh!4e-OndOfIy#^Z142~GZc@`A_CYa1VMI(FybW?9UF_XbbRY;Dp9 z#d{3sV7vd8B!Fr6xy~8Aq7Em_=^H}Z8AMHm=-z+_*`xvvDe-FhFH)};PMVt1v7-Bb zI^9E`n}3Cy?c$UqFss`UxK~EZlDTgZl&h zlXgV3GlQ(q`2S?HjZ8LP$ss7LK_UA|1VlVSTUP@gs=1Xq$^z@t$codwX+MW@Px{3$ z&4kc=1O6kNYs-3Z+jzD4qG!QuAY~5sNgsyocaPEAm!Kqs%=I1IF3Sb)-G-@$ z2O#@iPpTmFz?<)Ev388H+p(MSZ3SxfZfAMXNq1FmKp1yzzsb)znlH{IJ<=w^X%}<3 z9GQbz76CpHP^lxba$J1=F8^E=cjdTj>5(%=tTwM zO};m|Y_@dWLLS(PR(TH~vfoU&Z%ed!Q8F8SG6DXk_aiM8FS-(V8Yy$oqV%Lb^8cg_ZPtr4=v* z-H1CHmi#w;4`!fmX4_FD0^#=@|@ z)K}q-E#}Q?>w6m-Zq*}4iZ`#yK-_A?*4nC3NvDAz2B8b>hFjmCKUuyZdM(A0ikx!d z$$eN>oXCi(D?qFl)v_D!0YAG6+ zPj|fFp#A-|&YCW@>Lt@}`o3mAAX^%GA>tJ^j6?(3EVa!QGh{}DCsx+sy9%#FOv_y6 zj#$Hmn2m}8a3}2cdarjLKYbH{m|y%zHQF8wVRNYKDQ&{U&KD}zSWgD1LHEnkl=coj ztij`+eV%VGe$NbM^IAB>7#u7ihYJ0506`85vxr0<75SvAy<@$5-e+6Kd+)^yb`sG^ zPtssH4+x`x(@_|_q1??p#l9j;uU(<@;Y`@H`KM zp%-maN{i8nVU$9CJN;Z1l=jR3k5r+X865glB|{8K*tIW5t6o0ZVq7?G7k531h3CW< z@Ld#jbT5rVSfS7W480D(t~tAsfc;1r7u`R|6pzWCSwiAOl#(8S^gLu}Ls(wtq}&t? z83{@O!Hor}001XLr6v=rEGM>+AD;|M8Gj{=sP_$XY5qXr#>N_4rJYNQJpmKsX>cPI zo_n4MUF4CX?Fv)(s^=|FK=cCTv7R|jeG8ohD`Wz8WlK&4c|Pkgh9zg|PuojN_VCw+IJfo=$l(<9DHI`@j3eCviZLX zQkGa%4n9UPt%^P!c45U#{y{+f83>`BNYCo;o5D|ZdW4@LUfR=s?Jxnf5BEeKb^(Hp ze^Qy_{LLeG*hJpQ&K*?0gx*O_bT5X|3`%p2%@xMq{00x0K5rv|wmOHfphBXoVbEk$ zQ2m4kw&3TwQf2C_BXTA~z#ewFqvw-Po2G#vFs>WQGT0=P7@u)jj+Otiz_=z%lr*Uf z2MqTUs*g7!J!r|H^z@ZL8M(1W>@$Otk_3_%0v?2YV`vh@NB08PV+#udVtB^N`vj{B zM!V>(wpm{QQ^bMglX|>zkSv0YkkS|`GwK&3;G*UC9{j;ppuqice`1~3sPEQ}#1_7U zMj#m>i5P91al`uU$Djh=Q06nN+S%qJitF7ie)-k^^Q`)LMIz}Fcimxy_8G!B4#O3U zt07lS_pJnb6DYgL7$bp3uqlxPyUvSYo0pS`-^+Hg**_`QuaR6KHG)Bey#tVJTa>O_ zwr$(CZQHhX*|zPfUADc;wr$(SF5NnPUq|25FFHCSXGTUw##)&fbB#H2{A2!KimBnO z17FP?LId~Rcuw0#l_SGMWwXGm7)2e_3v(SQ%qI=EqFT)-q*jk_6lw6NFQ$? zOkBpCzx}pobBEjUI8+4Wm?_f8(rPo1G9;-N89^G^RHoE~-L<#M8*Mg7uGB(=x?nRyOr9345_k0gBiQh|0ZW2R3k*Y9MG; zGNBwOLs11J*?_Xb5+?Y7*Z**Ff4sH38DaP^D)pzQQ|{T@ znC%E{9t{%0#>Mw<`Tuk*wAj|tBtqBz?=bLYZ0jFW`TGuEQ2F*68{5=>WAx=$l%C)-mVixa)jnWPhW3|D1+Xf;&SYd-D!U-R_YYk5V zf{b-{{6lS|dFKI%&bU;XDGBQ_Z>T^`g0q0xWJ{(d!IlYv5&==k1*7TVv+$E6-z~Bi z7Ghr{^S;Pp0#^iR)>)bpAq~OoMwSvn&4$$-F_zEhiqV!aUhT2Qqil&$rq6gmo4#QR z;;~GVK6SE1Op!^#kZv`vc(%Qd<;59LR;hlHRtIH}b^^^@pq}X8(LQ&rEZYD(sVeF+ z#LI-f5&_Mx@3AF-x}lT(BUQp0*W6`czT*~f+)3%t0ZVHhVLZIN+|5;VTfpT_(sY(k ziNQ2n@zhYq`GRI^yNnks5#cA*FK>LNx#EEqX+@lCBLTmw($T>jh`@kkxqIKB^GC7XnyA%asy zcgbbiz5%rJJLCwdXz(1x7LxIDT{)=ufR(_b72lGK_Xe&3)+2tS8Z^_&_cC(b%XcGw z0j33>w3uqvde(Yv%idJ8V@!>KdU(2ioxJJKd~tj$3xN?sdK3uwjp+u>JsGg1_Wk!)6X@A~XfItV`jb_P=AiHt-neV^0fD-+&_q^%M;P$za zldXentX{)8yaA?t@w`Pt>~{UbQ>(Y5p?>XhRyTG;xRqT=TH1iEy9=k?$~5Hi zJt%8ifmU}9v*d^KX{POf<#rzX_h@>t#dCoYr&9yK(EM75*G@3IugV+F_M(JcF>bL% ztaTvK+D1D`@B8<5RN??!H1p-uxNs1u-R9fGU5ZwO|M+UhFMB%OpHg~@UK}{-2iziy zTWQx@+u{NkM;9>F=C-}`+ltN_GNx#sd0Rlb?Vp9Sbmea&?UgRhGO23s5ayld_iLL_ zdGN)B4_7Xr2s365D7r6b-A8t&Ab{lecOGgs_xm<~<#-Olb|@4g^}qnN3Q?=mzx>4T zIQHZ8M3L7|Bkb(Fue^v+?wQDK*!3lKKqCazc@)n45kwF=8A~?-RD%H!&FJk<)DPH+YukiXt6=Ke&{YWug?cH&?$6csDEs#a4lHx5 zg6q<*9h8D>5Q#e5p$Z@ZvHyDRc9#S46VCc}q5Ra{&H zj8+4q8m>*JTQhe6GF*j8oD^)Ykz7>5Ji+aG1ee8D(`3DwHRJV}CjJC`sNdjVj&PEu zV|EehBZ&zC^}=!7>F@wMo?clz^APT^*yvu-lOaKG6&n=J?Bu2VtNr+b3zXDA)j3;* zO7u!twyWsNJns4O$QxNBN;x1Lot9Dy3N=wnnD!MqW8}MIPT6g zdFzLiuub~I`XP30rRvlIX?48V1S+XM*WoN_A&Hr-qSuA0c7&_=0PjKB2R(W4HH~fa z7-*{AS?-!+YXx9di_yXCew=MYjk=#Kw~4rs-MsWniL8QC7WVjlGo-GwNUoI>6;djy z^}?}B`YaTCwYTkheVj+fOlSI*uswRCg_<6x~dNaRB9mro#xsh=8JeMO6~Zs=zm1ONWm0saUBEy3%jK)`@G#eFTdLWsBTj zXmvaSFVU^sL80!_%aR!PbUC#nvmj2k=niC=Ru;O*V;Tm-kWGD-lH%7F*Z%S%Pzmjw zw&=KIYl{4{$Wg&<4M&#yx?z1UD$6xw z4ex--$tBQsHT;b`XjQdaD^aQiTSI0`te~O(wl7Y(BqBYBh;<-jTclj35mG;qKhc;b zEsqLdb0v}lFO-vR%y$4#?59I;y|b*4!rKymyf#He#G0&w{4 zNfaMZZ3HS3B49!Ryvf3PyAfj7M9bjE0KRuQkv}wm6a=9qwnOb5 z{fhJDJK+wcq+!%XM?Vfi=`69!P#V;DLMysU?I|8I7wOTQ4)u`089XHFKgVhGE0pLP3B|ZZIqI%%x7E8|-mJHb@Bs&rhVG%OV#gjTHUZyJkfsv*pnz@dTF04T;~qj@pWyaZh6JlrH~59riQ3`Xy~ zQ~^6DXB4T^;VxQ4wBYGWCbI&BA%Ka@&Azw8``R`%j#KSV{iU_ELf&R(qM$;Z`3qP| zH=9s9a}BBz-}9aUXQqR2Dezz#!U1v}NgMHK{c093)U_!m`{ zO2|&S0E;n=;P+LbN%Y)R1ZM9<F^r#I2CPPdY>`-T=2z;RW;X!^G{*;{ z;&1e_{g^&Q&6t1}`iV$dg)DHW=$Oww{%n!MQKS?VqNYTqrwlgioZHY5JnBjqEj zVr>FTmy0tCtobDlgDFRV@woxk5~(EoM=_!Oe*9R(_|h;D;cfw1t-3`u&UZ8+jtPN? zIVLvG6dMq=*(7RsjTg&a*vVBMJ{Nz#(g@=9RSNshUGlabAF~Jyn0V7ykASL> zB13n=kepo3Ujk_M*_nxh$4Hh4@3@j{1gI;_TdhL^DL4_ zLzX@KpJhuv)_UAh2q=-}X3@P$;CAi)J^b$$0B=4Z7A|OScy2z)B0Kz7zEyf^jpE># zC_XkLlXXVp1ID1Rd1pgF2wUSv=9PrMNFVdc#Tg{M{eJJKQRdymqvDUo+n4V2rzp=>Ok<2{MMVB@$7E@r)r3%VzJofZ+je+1%JPA|3@RXnWRppCjks^*ip z{Z06g?n0}D1<_i?bse1idhIP4@@fW>9cw7zgv}yJ+%PHY!&>Y7DD|#JK@eHy6OV^c zY6-}jP@-QuQAltX<&x5N1Z-Yb)TCN6FEk>b=>hs9Le00|A0C_Fh)gDJ(OYRT)YP;h zLI^rHS39Wgjzm{TCvW>2fW-v#C0cU0h;-MNDf1BiqhEZ+Uj`t??xG+|$p5bETt5*! zy>&Z%^;K^=)0*->`868N*r0JcQ|*9EL7QA#(Kr;S?vSXemXVxBb(NTV>iMU2pxnYx zK4ZZom!dm>GO1|$tT_gx*g0%)AsijJ zLTFOY8WRf7C@>b7KBJF8Ut8Wg59OpoK!C#>;9a{n+RD-Poe;I+H(iF*iZe^{2{Zv# zs79`p7R^$ecz%7J%3u4tKepM2sMG!Qag3<5C(wq$D4W_Y>BZDp?3-Re(v5Y~|Zk3j> zIfGX12`On=R_&yJD9)(e;V8gK`IeByh4(A%3b{!(=ASiE)7Mg6qOjajU({$Lh`=w< zZbc%2he#jC3vv^j^hpj^hl=2ULr?N&JoTuJ_po%Bw@eyN4)ql;WXZow-f=;YN#UJ6 z>Z-3OsM>m}1`tjixG*YS)z|(eh|%=O&(G|eL>?cIM-B0H=~d?5n8&W{Imi80L4P=b zxAA`kq~qWhMmD|sKClWug2;AT5(2;1c3B(Rr;zlD(K13$Q`eFCP~qJv8S~MB4*B>z zLvqZD^P$!LrA1VOsT8%Fk3mDS0*W6X=x_&;jc=WQPvQ46E{kEf(rI|^#{h8mF@rH< z3gy2g7>-}RLTY0w2*Qw??~0Ihk9mEBD_U7ix^9ezXqpAbIJA-j5EDuUR@j`WAS=TG zO8`jsVJ$Ri>u*S>GKw;hi(h@~w0LY!Dt|6a?c`u{@4&P4dsII9R@dR^KwVeffVZ4X zme}lL=*9HufOEUPT?*v_9hC~s7fGoz)N)Bx!RPY?b?Ptt%)%fmJ$HN}T77=~1(L;; zjd;tiOiFIBk(MX@|H38w!@Ly!gKygm3IG6s2mk>8pM?h}OYXRjbR!ZLlG9 zU8p0BiE43d<9SlRyv`EI)(d0;$T~s+(bFojH6oQKs<3^%_y{MUm~xMyRnw(PPOoIK z56n3|?;nF!x)3iJ4)AsQJ?*1wQReb`J(t=I%Q3he#=?sECB)Q4GgH3p%1+ubaFYI1S`sDJ&UFzLypB_AG16{LTHjFGqifzhxA2I0I=5SG>N^)P;rN^ zGp;Ux@wcE~mt|Ql;KuA8I&b<`UxXR0W6SaT>8-SfO}AOMafKI9V9hn;@`vZxjJvA1 zG*e>Wt&u&wG4YDEAqFH18gy-F&NX-FHXA)4eHxTgF-2J9Q*3woJiK3DlWveL*f$}q zQk=(XPi%Y4N_BllI|Nzg-D0;kz=b!Q70rlFq1xa;2nD5SNdQ=AVhT2+9#j~{Gy&xC zdAjUJDNKXCR;LdEa9n*`W#EC?iVB;1*2tMdruNq`8E-bc_7l;8%;dnB6lTIjNPwFK1xIEi>GIHSDJToJ!@9eO6tQ>38Psdx#Wc+D?>oFA|xN^taE|gNyo|P3QPP6g;PW5mg*H;2NHM!sbM1*13 z#e>sje~5|lVthi<%t`J=urHaNdV&`dY!ZM1AdV(ORsi-k`CN=Rr{K9P&v;D~c&8sDvpH8ur#x1P+x^QO#l?Y# zhF(5yQ$hUlWsPk6X+}D-S7@#BjtE)KA{|Y_4m)5l)>H0#QW^UnmO_a84$)-J3gtLJ zZ|JmcSW+);gbhcYvi$=uZdY6#fwNbwb==^>5?*!bdnPnFR`QV({^roBe&ULxvLWf4 zGC&$=S4zP+8z;h+EirKVI8WEi)rbRPE^4|Nx~ZDEctj~puq`L2vWh=GdCFfWaQB4G zf61TYOJ{R7c(w?{hd)IRcgljuA7aTu#`Ma*%(`a%#;T@R;05Y2yFMCf^2(QpBdIpm z0QFBzYZauVZ&*#H3Y&=3Z{&hj>$;Y=zK4$~vfhC~lZ53<+JhTv2Oao*QgRfI5}j3D zB&Es!5<0Qxe4aM@{C!<0wFXTr@>RfzIjym|?o_B&o9L#qoYf`^F z9!{F2yEw#kwO#IrSTZ2Y;^o5MKmB9m&>40qW<}1#@ zaF>vLA0uPPm+t(ygGR-^?!If^J7!yOq_I8Sj+?sTjH|W5BXZj|< z8ThqdMr>RmQW{I#M*&?vRoP)9TWJH@Om~Ti{gYZy{lr;LY0bp@`u?&%al&noY~KN+ zl}GkNqslvQeQu5y4!Iqe`Ba_);+oJQ@H)ZB}e9Iwk2U!u_W+ z@K$U1{!k`Xu8Yh!J%_!wk&}z=yX+z52u^_vSk*O)?`e3ZbEDxERI*dkrig!KV{7Os zce8EMj>98WcTH#WG3{mOSl=-6^!UORQn$Tg9GV!!n9G}U&ZJ<@F7l}&62UE=`U~)% z=nn<(&+Q*r&VM}l{~n<=w%AD}W`d@AsI4P7aobaM3kGt=GRNegRhOZ*nm+W~T; z5lW{bg9tboPM%%M)CsRfzVgCbyqC@hJczn^1O|``g1gk?=)8{-?qY_xsBsDVIuN~Y zZnjv*r_N+h+5m1eYyA)i%BM;sO{rNY?9O1?!de89E@a3>>Wc-$_5S<@Cd5OqjY5B7 z%e82UL8x`YOmW<}wW}3(wh~My4|~^yvmLP(M5ueapJe74$j6XY20h(+U8liFY$_tm z_JV;{=88v?j+sZXBt#25B{(lYRbk@8aIt@nx7B>{Rb!OaeV|t$!sSe%jMEkA+o$I9 z4_hqOo5+s>i&7<_l#z0gW3yFCBN%Kr3KJKBG>C(W22v$M#1NGT5fK~zxUtEUX0#mc z#ST=1Izw>E6Lgalv31wV)Rs_loM%neg&Ot6R5%+JZnH?2i=Y!}7U&#z?b@>&%7XjC zZf%t?ts69MuaKVt#=OcZujVQ`A?zfo1TE)IvK5@LK&mq)!21Hp79c{~NMH#pMAW@e zDByxiMKy^Mr_juGzZkzFj~|YlmWH7V4U4@0@!cVMiN9NIMw>v8igPN{@~=x63MPe< zHC13_z*sc8z5c+=o10~U7k_L;RIDi@Zla3<<8t*X)xSeS<7H9sK!%;Kh>C4(R_|Gr zNtc9l;RhBXi*_;#r0LvyAHi6uHfUGN9Cjb*wR7p-k^>%?h{HTUOaVP3k+nq1sK_;b z?+{q?VwG9SA=ORS0Z{MWm^twRZRlCCP4c83|8v`Jmlf6?kGbh_&k;e!g67Ce2&s9T zQxEwy&d|kt0OEt1D4vTgh8j{L6%>FS2FP6xemPVEVgHb2Ieb^2Y~s_G>^dHC8;V_P zYr065JvRAAoWuQmxW%iQr`%>I;Nw7=79>;!HU1X{1ZP5 zUb(FJ?%0F@i_KX}RVOBhEe2Tqz$a_?olQ?xM*Hw;KghKncw0`w#~Ve>0r2pxa`nie z`7o!K1w3uvs!vCFT)0P=e|^KPXIl@*wzT_xL1VFBD|JY>^<8RWgUQ>yOR5+{>612- z_i)<`d#dU*azp2){kV0oZlVY?n)C!65yL;L;=uOK((a1Pb&JGp@5aWz_PH6`EbRMS zqjTwf8a+hsq4VZ;$EK5;x4U!Qi@yDtzQMwC1N%<8(Gh}DO1_#2H=>jTUo_ogiZMwF zA&)WH0U^DQ1fp}mC=b`rIf?WwOLGTgQy7K*SFJ_}dGz@ZaWJ^FB$3?j_&@xM=tOZ= zs)d1nclJ*{_rmy%7vYsz*jf2^ZEC`Lt_RKof_8{g?K42gIa+`eJ|8;j|Q)6<@!D@Dc2=)jL;^`Mm zDM6H5acyx4YnhwnQR^P96?PvR5~0KIFf(y7yBRJ6XBIAZ<8E)X^x~v&{Q~}X{4B(# zkohNktNg@Jynn}NBM)a&eJ5u}OFMJ@ek2=7C&qJ#uAmxhr>bC@d#QGECzrk}*pL{2SEl{F1#qZY3|3He3~QM@Ey zNj3y8h*$U?iy<3YLS?KL{T)-9NSF>u_d$-yozxrp)4&M7268(Cwa~&AOck+Pj+RcUsGtp>P?#+>QLXewa*x|T>3GOcjNKSXYd0jAKem2}i%>#a$05l&I1wf6Rp^i#DhcCmsDy8?qISqs_TEtM^A@Nohp%>=D z^SVOS!==qVC5E1^EG&9Z7Geuo?)*aW!^k{dgx^k&fiGEHQHrJG+=iOoTR6Eg8^ZIt z5UWHZxlHes;{_haxec~l$gta!mJXSVA1nmR_3lU3zuVIo+WNymer?1zp4H7Oxk;PB zktCI|R-EcyKSl}7-CO&+5n5i0T6<zLTk=tEuC^rqlm` z+q>eicG}=b-1(%gyI0L|uB;;=Z*)uHwa*#(g>;g*ifV1f#G0g}h(be{4?zd0HK)7f zw`0d50YpNoD3_hlg{6vz1i^y!c@5TU{Cv8DS1&Ow+TcF!o6h6+h`;LSoHd5ZLZ0cj zT%XQZFJ)R1m8zN^44sR+wk#_0sqRrFnmHCu4SE|XxU#79sP;zlr2WWrKxm}Eau_k2 zN$n%zurQSrCx`-_YcbTwor@a(E)K(Yay%B}!DLd~-*L+|*^j!yqb%&&H7A{+LJJIj zEPndq`;}QcAPe{a`pJ_f*-uPsa!&-`_!bYziW)I*h;&!piQAw*wb|0jz1J9ZV$R>0 zD^pv$Et{>+FR!Q9m7~|w;rZR@ahL~qILnmyOR{b%NgZ92tTF#BlDdI^iE8rEc{;#w z6VSg&sVS2p`al!IC(}q9@b;cG&e-FR)WNf0D#$iQ!PV1@yq>In&Zj$D7hhH)gfB7LzZ^nC1y4uI1!(12P1+Qn%-;b?V6Vq;S$`5sI zd)7gkVY#9}YyrAsWa3tHOROjg5vxbOSB(nNDyxd2$cc}a$cLWC3P8Iw>_H!)vKtK| z$gLZ5Qxh0CA$P=2ItSlh-M|-WP+^u_%@EHM9$1w;K6t~C@EaoYd=)XEo_Wjj!qSn( z&tIQj&zF&1U72+!>Nk2iGOLG?pD#xjD>nD{fdRVQJ#Bvs(`Ke-#)hUqJoPuzzu5V+ zf9vb;F#C&C+VRjTN~~G#G5npi?|V86XlG%lbEo;KVHoE60Br%-kGas#!p>3WQEvlo z>(MOqI$ti(8<$_bwBxo;VXk*%!!Zm++Xpp8?*CD2 zhEIoU_QaP=9N+9z@B{P)s#u~|Y60-e2_ti$cngNNlhP~a^78812BEZfMo9#Q zD-SR_n6ntP&_ooG#A22*f?!kzCG{U8LtS-ZX5rC52nFph8D0how>{Kh6ui`rc*CKf zw3ntxx2~5mWTXZgY1E6fkOY9iB9MVRNRQ1gcPpXZ6CpJvurzVX=%ryP1-fckv)ird z{^@Bq+2-vxE@ zLHWKj_x%*8olz~c#eRYD>h3u3Qc<=`}s9*v#!eliGVr_&p1fw`V88S^LOrzDx z&Zvu#RCYp>2)C}1i=IM6(_H8QDlDJS10u5B&NN1s0+c}8D6AG62S)2w0ajRDEw-F9 zT{v$7K8D;T^oHbRsC4NRuhHqS|IR&*QGnRHQ8BYNkYa|(J+S2wHeRVI!=RX%T$|Za zaoKJ8YvMErnQ|$C^KE=|u^^l8kd;wh#~bWD)uF)@*6ZZH59Y!^yG)?`nc}qu@{^r> z;2_q7!_2(Q*Ch($SS!%H@UfuWpEkWaCFk_DO>89AZl1$y$MTJ0n(g#iNa8SPTivBV zH24F@n?x&qI)Q0T+A>B)v;(^y=~sHbA^1eNV8K0EL3$g=T{6I*0GBOR=|(_Pp-J88 z3P-%*+y#S+dfCnMRa05Gbws@X=|lCL`R9 zQt*+@Nqd|zM%=rHZOBSZoD)tCYw;|rz53lE1Rc%^JHldebwq+a631O4Uabc%th)q&w^RefcCaUAaAWG0zy}YW%j@NM?4}Y>QQIC(3!?~7nFZP2&W1(}BU-HNF*G7R?^@Tynje#h!%M;- zHtgK=`EMbJtUh?}DYgZbND*-8nw|Tn)S{1c45N96=Ww6FI?NY|35sMnxYrNNgIMhT zW|nnZ7%b>V`QTRSz*d~yJ@?i1DV$>{B*GvI%)s;^DtB;3s-OG^dupbD*yL@GaGo<~w^nKVv~(8~yi--b8e5 zx`b-CkrSryV3mU`0jcny+@3>@a3VW;xdNgUZ(N8Sr56iIsmm}B@wY&irn>(hL3~`N zIZFnNP4S!UIYPIL50EDQiDvI#UmMhfsgb2o;c!YV6h$wa`Yuo^@^=vZUm&1y*ZRUM zzQ{m$*VJ(@H=OGMcT$X%ylY|n7g7%RgQJ@_FaEX*H@8|Z5r5;>4$s$6y@Ey-C;OPW zw1dT`$GwmcR03Iw2(MeFv;`Dh53 z2Yu}dqT@L?a+D;lGBAy(WUvmAW%N=lGrfzPQvLzDc|FM5Wp9$u?GL0)Nzm;J;KqaJ z?B6{{TIA~xE=3ICt^QlOUB{CaM44!LNj%$!#}U)~?*IMQnhCFKt#j?;Y~`@F%gkUl z{jq4Fimmq1QZ<)zy6(qvQIN`X6Na1$mcg)=k~aJsFxOwZ1cEvh0eOaEj5apLLt71` z3`f#s1&m(~FPM=0-QwXD+9GH~@v*&ZThIKS+0K$;T$w<8&#hmC^J(227N&WD0ye#| ziN`MaEaR}F{|gJk!#uUKo*@1hClqh z_&d-}s||+x>u_8sL~TjNe<@Vh6Oh%2 zTrY&Q|NOj~D%Y-r_zyoi!S?lYdJH$$*Gi)CK^yFfVARoVg04tFh&KC8OXCU#mZmQP z)H2##%O-NIjg8YlMZ?8WF2=7vH3DrpF>N=?=2+ZCHCqZyS*I8Gv-o9HM&fBeRF78i zsYdlIY=ARdN^i!)=tPqK7=3abdSlBwmr^&zUD$5p4DVf<`&(#(xz`;n$9-X&T2@1r zj{uQZvJ8CAfzRAJQv$UCA^de;#{gc)lP@l(3t}$n?2bOVjo)qCewR}1IlA3}<@)HE=*AW?trdlepzK0sG8HOxMFSCHRFTaBBk>iK;;{k(0dlTJ~3 zwJbIVgH~$uSD0Kke_gi4wknAF3AD>VMbSo3Hv8WJ9V`@m-Z6HB%BTb?GQTHp9Jn%I zq_>$_fCS+1!95~E?L9MT?kqaWX6Po5(Zp=5yf{&O9-Ppg0rS^nbDg@ zmL=^GRE2v!Z8u|eIf9`eon~9*m3USF3E9wvwzpqNS65&%3=Bj9xx9y7tjk7`^**;d zyp>a|uxa|7?9*f*%l1Ycpt9sQ`4a@l($b6uAK`mnxy6Cp^s1C$4G2TE+ zNyrlVmO2^D?kY^<2(JS|%%Qn!pu=o%84h)M^usKx4>r3KxAl?hIvS)g)*GUOq)n+n z&3pzZ0Dz?^xM!D}-h*trkrtwL?7x(0r>4>!0FAa0#~TL`yuu-f&kH3dtDY%OiV7?c zKh3bH1R1)7E`^k@Rc_Y^D`#t%Dm&M%PzMY0 zw>?$;N^`|7VY`5zbrYVbFCV!D(ONtvuMp3aNxa<@D72}QqF>>uomQ|RBmWPI*+*fy z(8@wex&EM8q!t0X0dy34k^=8R{n$d1-_V9vI^ViFt*z1~#7@c7s3;T+6_QW+s#84a^N7j;cM!D_JGQC>;b zMciX1#(N?M*QZ>qAmp0$3s~ot(MXiSJJDo-IJ-O;R+Dme7WRlm0L34%iAFs;DJ?Kt zinfmmT&4U!M!|5QnSTz{jlh!M^1&u zYzFIy4GRMY^?q_fOup?Lfq+5O@{OcCqt$CSe<-ARj@pU`~xWfgW9G+nu={2OoN8NcV{3ROX~CKJ7x0#Sn(e&Tb#Fa~Tbe z^M?u)iO;hZqfS8NP4$WcZ}*2ggGNyVS;q{S#HM~Afkjs@jK*fFJP7P+!K*f#u23cy zb8YHw)5hxWyyd$VGViQ0(mETm=Uld*CavLuY zJ?sv(KG}Qkh&;)XsKS)-P-`!)#LQ3k17|THJfhUj4KT?D(`M z(IJXW0){DQZ>@urTFttxL(NbzzyF))LSs|Y`~GvG^`im+VEy|-v$XqBTwH8_YUQRT z|BK+Vr6KL~(`MEEpl+}Uonq*5KE}*P(lSpaOC!IJGY<_$l*VM6XaF!8`Q;-3Kmv%6 zm?9<1y%cTAgVwRb7ogf_5yPaV8Xu9!hLhqVogu*Ky(M5mrcEFZ6Iu4VwPOcwAyrZE zNK%AD>7eY?y}YZ6DjyMNmANk07>Drfj~e&y#c5VsELyf-%gB$hJjh*%c^x_Kc`@4= znoM}4uzJXV+6@mT@fBJLmt)7z9b_``n2(!NkFj%T>8qrX0ceyf-B;HX)x*MU2E-;h zjeU^2fu&mr=Aj78qioS>GyZ$nnR3J}c9gdEi+%oL~My~tw z<=0Ic$g=J`xS3{CNXSXW;I+ax1t(yU0QfG|%amTQsnr|CkXuJOk5cQJMZ!-o_sTwFK(blIxXzY2W>p zBcu#4lP4$wz?_x^lYChQQo^6CJ~L*H*EwcV+7vRnziZu9kF|l#_$t?)Lb0^8g zwv!M32-C0D4DKB%3;&Y0c2*0yw4-v>s$A&ZsPffA!%gEc#FsPHcS%mF9P31^dP>6& z-LrW>8f{-o&XE~*l_hb-BdagHtf%B~ox9`UVLiHB{Qib{RSbu$HtIB*~4icv0a^quEp+LR< z-SSPY)_nL84Yyc6txWrY_pn+2sejJFTkYGn^*Hlw?=^Mv?{youJwPi02mqiA0ssK_ z-{Hi{(A7}i(9zM*<6oNF|FCe!sL9%|3m|m8t5Mv}54N}D}oocILJ<2bs^ z@C7k}qon7oLOoLuF6GKk=8Q8Bztyl53r%ZgqW)p3Na)F<@Qj~_`5yQ8ofEe+M zM+M?U0i{>z7OWPYd0+!-GK-VyJ7Z@IT9&uzp%pZC*N?FIHNuj~FPf)#nK>Yp(>)C8 zm2QU|jJrw}*DOF2sOgRUEV@1jZ>K@z_6ZxPbtKMd7Z94?pn^n;fgbym&!Ro?o9*y% zz9e(JP7`?+%(Yn&%Vl3QR|@%N?ql#k8P+*7U?~f^ajvdcU#!?s-wlf(iSIs(MOfp= zKNp|s;jtA2k%ov&0nDZF7U*^ukjya23+0R+4VvGolb`$ox}X)7#v`{DcPv)Q!yDih zoDotsr2`s`=FAxFdYD#%=l(n!tNH2uheJSKezI^!H|2F{fIfyZ3Zs zUnf&*uPHO+G+`xKQvVDT4m#?iQ>$DnoYgUM4g;P8+5X*cI6|$nqCRNx7RrB?kl>n1 z7t^$6yfQ@kdgb-{V1o;nJ4i*ZKlqxos#bgkxbj5!G48B0x9yy>&0ZnPY|^CHvi_;QGi~{w(+V|H(^pDh{NG zWFt}5e|pnyIsSjL@_)>4V?!Goqo3}ue@*dxLxq%R6C;0(6j_yrqgib;VG)UK zl<^o3KOo?braYjC@lNv`g!GoffJ>z7j|v8(B##Z`P~g_Z`gI=x{zbHt!3+5d#3Vfs zYQg`Vv5-y@iNAeZkbW^K=+ZoqjtQeVs#btPKIr!Z_!`jwou^xvNr-oeRZueylDw88>KC$LZaht+*?@Kg*M?)`+ZmmxpA9%sGSo!FeHyAM%xQg_L0lN86eZLwQs0+MSY zH&K651|1Qd>xD8W2ZuV~71thi3_iqOfN&%@Jg~nbhByGQTGl~y+sxwC0`9LKZv~^GjkJeWkQ7{ zh_?G3XmZ8@30ho@sJ!3u)78FsQL}X*#I)Mjp141hI&(+jgE>qwgPd+3T;*b)_YQSm z&&CWeuBVnLNw`{qS(59UBS>*Z2hc8kHMp72Wa{5u3H*T_`9Gu!5cBC z)?jgLqCMa;EaxWlN&)uG46f^0w5Sy^E&=(E2m}daSGY`ea zsIki91#-OgWjZ4_1s8Pfb54pS#uS0bW5wn9!X&X262HQMo=gSo@mVczPfPc1CBI7x zb(^aDwq*(56l*B=Ig7WuJZ+p4r}+qP}n z&aCv4wry0}wr$(isonjh-_v_vSO15I6>CO}Id0N>8}KhsY?ln07Ro>Gk}5YAR6hY+ zvdN-9ChP@oF_;X=W#zB3*pbQQ?Od{zfS57TG?L6_x~Tkvd6i1Gml#^IAEuYlKfNij zBOyo}zyhC)7^m<&IgKjP{Q<=pWFbR7hSiB#y%^k_az7 zmwC7ChL^!4=#&%Ve9s(VRp*Wdy<+N>P7N$QFSBE(c`>)1Xb9g|7pB_8Fqe>&=%>Qz zVnee~sUXTC0O@Z6^rb<(mSey!K6i1A;2(!>DD`G$na2`kN)Pe_<@1u9K{Zk+3u~)p zJ*arZIs0Vr9Po7VJ-95^4=3GuPC!7ZK$RSM-vtI2nP-fv;=DuF+ti5W^pp?y#I1aR zcaw0j(e>+_@T2)pEkC5cChx2rlqj-9w#&6wuXL7{TBX}eKcJP{O_*Ipni((`_o@|Z zhUuuB@=747x1SN5_es*0?Re@o{u+<4OS(@4r6FWNl#Dne_D(uEsN+x|WlcdhY)~ls zz}T_Cs$jx3ieyoNX-Q+%=Xmj;YJH-1BgS14Y?A5yCLx2^CCu_`iGQDZvUqj07;bSg zxEVFYBfWZxSy*)c@dXv~xh@(6c@6u;2`0gXEAwTQiq1sEMbY-+2+8aPttmTmxOCG((3r7EyZaB$UJ~5zSeYXeojdCtskqKh*AzIcNB$o5;MO=Q?<_icVi&vv-zvOC%(^ZC) z-mlMJi+jQ}5mJQtYo}d(s&QPj=cM;cp@gpg@)6mr6aX2-6dXEV@KM3-LTFM-!%!Ou zqcpOgEXi=+JHStLp?88xhEk$dN8s6;%1ONAxusGLRbSiLRnI+wP5miC8%0Ctwbqf< zzg(!@wN8DkUH@EUanuA$Lb@w08>+8z@*7EgiEz(&M3LXJ|h2wwGFH&FU7i_Yt> z*ASUu)T-%tDsHc?evDJ(DGH2f`-6oxN4`loJt?t?&+3k zSX1hN!n7oGV}-c}7MLH14`@jlZcO+53{^|0E9L~^jN?(sR!fM)5l=S!kFaE5rK{Sg z;q<$mq7WyvY4&)a<%~<~R40~GOWY8av3;@pvvE$>%vo9Hr&=+yF`dfMk~;L3brWA{ z-}n9{4BU^d_zDh2GW44_Zfgn&s)I8@*aG z-m}e@)<(CtFo*G;uwY${@rJ&>ipP{CojOO&B~IJ<=Luj;%}RyI}cs?4|V$@(aT{y5le=sRlo ziL;9b1*eG+&6y-T#7-zLyol{yx%-( zJfZME;QyHkSFts;SN~lxF(U&3;rut5(8TgTDa`*N5ngF%%U`k~`OVbU^-CAjsBHmt z;YLD633O#UQ)MrN!w^l7CJpWmmS0^q{B(DdN@tk_7KC=QAVy%#_&5{bIQ3D7IBr*{ z0S!l}!DGNQv&woxq$z?2F|-LBsyQuF^~w=LITfzi$X}3XYRp zFo4epTKwg*?}((+egG?0)LGEW6geHgcpfjhf=)OI@BL)^7c-GMPUdsL#GMLFk!2_a z2ZEg>m_QKVV|53mn~oWth)~^d&+Q;i&gXfVM8W5Ip0soIX#zBbhNk3OjOJ6b*22vx z(zc7b*!DsWnN`&1Esn^{$UiH(b9b2z3s8Zm5CCKZxTR)${Q!(C_elg?Dv;`%`NQ{gY7j9?P+&ZL0ae-pfu2HaxV(Fr%vM5@#6|qYcdm>_X zJcY5~GrM5~7CJpW8*}b+OJ{t)#7XD{&lS^HTM(*VV&YGTH<84)vq|?3A*;ThlVPlR z|7Q@b+;yRWmJJjfi@dFyGG^8LWw596P(eBJDqPZS*D&2NUSkF;46-Ykp_*}0i!GN> z1%#`W?{s2PPfhW(iYU80#Z*E#Kd}mUj7=8!Q4Q`uG5I%3@P^&F>cS4hmwO0bqIWPQ zP&u-KKEmO6;O3##0*=W===!nj`Z3pLX0^l4^6G+MYrOLp*c+nO>5rJljhMur`@jhs zM%(I}<)!&xYm>m#VbqV)kxBTRHq6>8V<>O^kwuHC2HlEPtaP$YSZ@kXW ztNYqagHNj#FW4`eJyn9Kd9TKEE%P{it>=!?dhU%cdr6Onuu*yN zGASx1jO{y|u$3P-o6S4#ztr5zGV=`s-Jxdf6D?0B@m>^Gw2l&xaAnzK@HR097s8)e zNo;!`J}NkV4g~5moR%76)Fhf+5>%1Fb7E{0EqT@*@zlw4JdnW6Lcdr1l@`Z$1P;FS zT;X#b)XL^vU&l^!FfT~E5T;c3ROAK{v>z7gvE#&ly4+;YQHttd_m07@NpFs(&s=hF z61S5;5Y`J8D7P^A{eN`=4Sk~=1}IFZDL#XA`v0$@)_>Kqpn(2MGUk^Ctcphn1VmQ> z1Vr%Pd@RfjT}<8oe^*HVgLCYX*V1`ItYPPgdU&=H=2QU+SgNJa%d0s`X6^o5OVO>t zv`vAKv&bMAC|H>3A@14zl3?RH9hf0F8p(BV2FHGxJTNEcI;Z>ibdP+^t6OxqZ}7>y zVR+SWqH6O-^DrX~jimQZH4bYkLL<$KN^el?%#!Y!v#y`h4y!ku@!> zFKhLgB{C^~oz3winlMIkzhf4-nU}pJ|3FlCKc=8p+Q`Z|0)E>G*8?AK#BNvIasm2_ zH~&nVzeL}72CtZYD5>p$n;F}w3>vS)c|=-J@1YiF|H3BAFN6Dz)t*tlb5%E_FHP(h z6p^jxbZ*hD;9!2C!Ds03@7eEsUl*>wmx+68eLGisM?(fbMbrlKDT+*5Wq*yh^ksDA zZ_)IF(Ou#O1 zz`jFb`3rUT#R^SjvKd@bUQO#m1fM=$X`%+`#`Lsw_rbrbCf2`6@}aLJ@MLs%?l$=? z%I5i9N*)<}$pZb|>~Osyyj(A6R4caoGTA>xQ++w}AAW%gnX$B#(mj!X9%O-JqsaXU z;J?*5A0d3hO?(=-bG9)roXOJ}ExUSGynHpbf>Z{dYD3`)$PUXZfQ&uKuM21C%+LQk zei;n48`eV#gwsbvc-bM)Uq3O&NX5mNXUJEETpY)TC2e8GUl6r9Fo1wYN|?uczHIdSXT_27gU0wwmxjaEKBB@ed>zKZeyKL|)qj$|9= z`R)DApIg;b*eGo^Xy&_3e=QcD!>!sm#^lDq!5Qcq3hx6?0$U?lmh0Ks`HTB++$Y8I z@Ed>Y&e{n^@Ic|t<-^xR>n(8voWh*7@qQSegr}YvC&30g8jGM&j6n_ zPslO>7OZ9FOyI@E5FbbCIKdQy-}txio?ZT^KQ>tn`&%S*ju`(G-pUG>t`OT8jSIO! zecUqI`X-4x+B~I1|J}kZ!2P)_Ye)|2I8jpZk=fm@><4O#kXk!#YlRZ$A1lYshsMIk zo*F?{RO6H+plq`j{Fq~?#SH})KzdK9dwOXPQMbZem91x{K zBL77N4XsoQ?E~=7LKt9t=5S6Rkc@cpJr+4~Lqo`ek-<8v>m5#zq0G6A%{f416>WnRtVVhq&?Z#4KE|+<& zJ;f%-Nc<6;jilymOj7sp{1_n$$bdA`)o7ZRZEkoUs-mEJj!f~(v83>dup{xTFbl1#bu(7EREGm@yrHksV|lxcpW9QfRR|qm7 zX1>E5(86hDc`uxfyfc47C=C!G^V^bZCyKscvLPbb_Y#kFaSqdlwM6Xj*dZDm6Nd*X z7q2a(c8?Yvd7(y?cX1`CUlmOnD7y!T%;SOZ>v)ki^Xc4tg8$ zPBj!N=Hu49hS-{3KW!pzAd?#(vk5IntG>i`z93N88eq`eRYJwqLLC@%z&HKNfQJD~ zcIdB-tgdvF&J(aSFo$F~*T@#K$K^>=Vfo9GcfOAt=MXuWOP^{Mv?Vct;^d9mY-2ap zGQ@6w{mC4{lVHwrKfw}->7ds&x~jG>SlA5t6D7r->1c?c#ffn@E83N`zoK1b`+$y zcv$+^wJuc1tioWDHe{)9|AXO=7=E;h{A8V-M3?AYKs5u}$HDuMjnA$3zDN6WikF@P znto^xL_VZbXZt)AdYKA_x{qdP(ZboMWEYi@gt&!7D2Ur)1aTPTzXy0MZqfD0u_vb-6_%SxbL_h2xop=NiK>^Jw(#E=elbc=E7$}@1!zYLhX*h?{)C+R zTz+KRP=Q6VkuTz3`v;J5{_FCK!Aj{sMOiZebD3W8)bD;$T+62|B|Urk#1T1uo#|$j zpisQIa>>)3KrYeg1`hC7lLOQ2nSYqlY|+O{^r*dFD0^wUtzkJH1xPYce5i8l`<+K{ zQzP|4QBIxjy>xjZ6<7D-Yq1FvKf=w=b6a%TMCR*8UqNhUVP++e5a2}?s8egpcJ)TO z!Y@i_Z`B!nw+(|eg`i`=xxj-=jutv2$E=B16&%GEOOF3KOmB>@maUnjC9?L=ml}AZ z$v2%`Crw>p(7Gkq2sdCBV|z1X7FmYTXHsEM##BuEnWf3>b4>)=d}8Lz;S+4HY{-yg zC?$#_V@yfx>Dhl=7s>P|o?G_UXMwV8g8GDF{)QRxFYQezph`4GI}2X##(491B%LAyHz27LzY z57X@jvlNu@m)}&#$tu4fK`SsTCvZoCIjTI}KKYvHBF_JdFvhVr^5Z0=Ri^7!Kr$8U zv{cF6OkNs01fOYU*4TO`r@-}>LZ%C!e5q2G1;hA<#dK7G3;>uCIifArs zb5&?HmM@KxdY=^@y?(D|OgdC7ICfXO|Oh!1*(sF($uojeg7}3X-G948j7Zrp`)z58#jXz{>rsSlI;q z>t|wCR|Flhlcei!hIV7w7rYYfWd{5YNyd}0<|^NyhL|^*D(&VuFNvqOG95hPDw1#{ z#pRS9jzAK@*s`qhb|a=^ra7Tnt(0{d1dA0+IHjeyXs`=oh`af5G{w@ueFmKn*Kkk} zH=KgvW*pH~Q116n{y!rcn;!N}w%!$Ho1&MNU~V+Y)7#y9K@S&v&SpFUQS#=-KRx?+ zVU>e-cnde}dX(x&j!Fs8Rfh;lmo=Vg$Dd-ROVt)x#Lsp`olxzf7fmo#DWZEhH?NHO zky?)w$Rlbl1Au3w^aVZ;lsZp|{pt0g==UFeG&ojS3DrW9_$&9PYuM}aq%4HacgB>Dnn-Qa^# zXn5DjU&XGw<&@5FtIe2kw{-g2*{hp^{_g_;JYyBO_U93>tWI{+Rro> ztZs;fq`}d4U})39rO|vY%@i?Tqk^+&!6aKM!tn*q~_%mD@0h_DNy2Ck`rPiaJy@V)wzi^ z-P=OcEiTF5$hHQh>_yhd?ra4-j>JWV_svoW)|BPY^%jZsE_BF|`4EC04kIL;2aBNN z_UvKmQvl%#6#d3>u@}Z528%cftl_#1)wgvs7fv2?n$u$_YeyMjp4g~Kk>a|!RK?bv zMcX;d#Fw68qLyzJ^H?^Cd#u#RgX85*U{Etvduo<+mG49rM6Rms{+_WtJR_hUvD;rf zBiTzW%fE}z+$X?!4FK_T!P(|t&nZE%p#E+u&kirzC;$X7gKGVS_mLJ5+w=|kR{bY^ zn;xqjGivKc=MOQP6P4@ge6rvzwK&6<4Yy{HCKz___(f9pWl8|={LZdk^P#&!S*PAo^i0O+?F{mOW%$9oBU18t+d z%}pmIapJc9!;2_BV*NPbi=(L6+n42c6K8)n0y3sQs8d$N{Y<;tp?tlc$6AwcSqevy zuPc7u3jsluckc8uvEZDQ?FGW4%@|nZb+icQ_m-*Cd6wGCG#q@b5X&YM!-XZreLHjy z`fLt^jz;g-Jj2IVcIt+j{YHb{qu!G$3>Lku#Uk1y;BhRJ7sf^Xyi$5HdtS3%@pj4g zbayx#BA^i)UgCFaF{^|l&y}|{8k=+a@IpKMdV`E(=Th{aZGC*T%PRXs<{fS1&O1`0 zbW;7lB)-rEEC`>AL`8)fVV?Il{)`1t*RnC;t9^6{K3;B|0h1b+G|@OxjY87~8G7}j zH%wDNgJ>o9Y!BJ#Gev(sOf!Nh$nMS6-Y<84#&a8_v1I+|Ko=3k_Ja=skHpNxL+hyw zs!CKq?#%?|O&_qfrHzjRcT?c(SOX!6PNUxHU!b)d4qrXNJD(i6gUxf9VR0I~`McRc zCzKI(pi>7-CL?}xRWFOHvUySs#wXDFdHw==z=;a@#kerBYoDe?#7DOshUZRJvt_$+ zHZ0@5<-^WU=eJ&}XQI;^0{<|gpTfwyiwLXj7(MtSdcIT{W7A3P+JDImex8iXS7R6+KM*Y5(HN$xK&F~2 zkveULMj5e>k&;we@r0z&ut#g_GxSlA3u!=`J7u;aURBgB(q{2nnMMbT>htA%v<|~; zwpls4ndEphFl>!>=v|zrH`;tmZ(qN*eQl)r!h{q%r zH%6ISo%4J>W6CyRJ#2Y7YPl9$`vpgv`HiJ%+s^J!kgY}6!YWsidj)|zz_nc0nln|4 zZNftu*{R&JF`)Il+_XUzj;yPzcw2#O3~Im`H-dZ3bqebb07kq8giaIe+&A?x^!H%hK*e>-5b2k*>pTP2b6iFFni0im5%vCaX)mc63tO(8%*KP^MEjuOxc z&99C@#U5P{pr6OB(LuX;`&T4NFUM76G`|Q3^-JPuB4kwyqY&OZgiM(ZKdJM~MY9aO zAb<0Hzea|5>~j?OT~cn4XL7Kdre#DCI_W}$5_4395`^`h#1)9G)qv^AMkAY>qfY!g zHl}ZD1l_iX(7{QdGtF*J=sKc6lNg-5WqsYM3A97Sd}LUB)UZ?ue$A|uIm8BiJ^T?q zU~@tDqC;;v56EvKA#+(f%fI2QG={z&kA{$9LaQ_#hF_mqQ#p~w2W)_7(sQ^MQYV-G zdrg^k%^}yQca9)W*-6L)(C?LgzDV4|>SV~@w$Mh^BH@d8GUS-xLhgMXzUq_rNGklM z!+=d?xjI{lchxc4IDWC*usbV}^H8#tut%WsG;%4G!9f3cTmMt%>4&fGP7%7$8M+@A zhSzu51U0VHJxG^`kPt)pcRn2@_BKkf9z!0$l?mc01CFFDZ0H_3seJ=QHQ$-1?8vce zn3c37MSH%~QlR=2x*bQJVCWfwxoaP!sy7t zZ!)^8|4&{&=MHMGO&TMWX^C#Xqa>7Uo538jDY135A&)b-C8q$2CHfA{9kzL=6ymJF z7R8+d(@Q;&v7Z$Zq()Zw2FqozCROKup~()tE^xR_+S%W8?-h8xom-KJeE) zY~recJ3^F7o{B}UU&2|0t#XP6E$1FFCnBFjWjF+VxCU>My>qnonO>$+7}=)DQ?bnc zblyIpB)<%$omxez5Rm{OgHkkK$UfHS1CT$f=$B_NaXNt<-`~d;i7a{hJI4gBP9fYu zn;iob+wPE=M0(_&3BvUnWw{9kN)r8E&BYzodTQd`@ee}ec)#oVqD!g3y$5bKC(r{V zrG&5tW-{gYen~Gj*)BgzGKfn>5pa@kZ0pLqMRn0)&{4+*lhFKRp#mC6;F?jmO1|)> z<1nIO!S%=h8m5|<_g;A^Zx>T`XAifMJE#_;e8f3Th=5cW6W`o0fa&zq@nnY@Z6_X9 zzcLnu^e4H6)!Nxc-`!oh>Se2ME@$iT82?ppj^f~KXx8xpZ2(}{r*zpm3C zV>giKP@z7)#@&4ivkiJZ^BJDMshWhI3~9N ztI};GS-j8kw8zNd$hM?pay}Y?ML$RqGDqT6WH8n_FV;DkL)z#{EP%L=p^>59cw+eO zs-SwU-bXB%bL5a_Jnl&T?h?aj>XeO4l_e@+R|wrRU_w0-#qwr$s4~oJ>XN&Y*NYqL ze-EszugYuQ?wNjQ7$TZuq#+!48|C)>gyn8c)n!VZy!0a`{7B@V^}NE5KbAM=ej6=; zuOHtqg1%{gJj0UMu*?WSm*Kab=dTM_VQfP4m!+zP8*n9Kf5Cn1xc6}TMJ$FN#Wi`S z=@&>G-g$I#&6{2ri(Lf=IXqF6GIjf;giTGm_|Cqu=1_1>CDk^jW}04! zEfH0j$iU8F&yef;vt|jMd`bLZa6RBFy5OBmgcGVm4LXOv+GZN9dLG;kPAn0Iu)Hb0 zY$0?B(Z$P;;QU}|eKth0Kf!mSA+Rb zY?gEi6_T9Lvbu+|X`gwrSvnb=NtW|Hl{vO zgd8g)fv`PK>xcv?f&%4T0pBixup^gV+~*g34tl-- zc7pWw@)EE}Yk%A3#Wj#HT^B-~URNp6!%RirIrDp4rK>BUD5$fXCrFQtRL zA2E&4g*z#1i4$>Ix=N1H8Vc*A)EQ2~|BPY&(~e;uw8T|>%?4c^LiP|AZ@v1IJ$B4` zKrS@bgwQJ_T6a`s!l?#AeaA6%=Xz#DG`6iPH!j@GKqkRzGa-QNK>)n-*8ZypyD5I# zaY#LA6fKOGs)-V#W=8ygwyU1U-dp&F%NsO|!_qzn%znEQx?8uF1UHVBKul1{PY=w$ zI$qhG?xLKqyYy~ME#xEWweohW9Bdj9YS*`M>;gxG_B^?9UyJH&z8UP`HNg#qt&z%h zRj2p7S}SlWN~>c@XYpB+n!C~Gi$rQ%-6U4bG-bo)SWM8{yO73~!|YMk>H(OfJa>!O zf$$yScU}2t(YXrc)CD&Ni&)`qFyfBd^Mlt%-PuEvpzvP}H8-SWYlBg#~j3gOGrpfCQ-LIT^S$f+HJREKU_U-SwcYxZg+0X3{RkDC9ex& z`Ej8Aw5_?&54ORoN3MCf&UcegDZZPAB}i;Cv}1)ApLgX!-9ENzYwQ~9pZ3g__d(K< zyYrEenL^IgXHQPJNWTrX%cWQzD7##toP(I^ksBo}UU>Al z)$9RBa4exyoLIT~GVT5N^}`(DfX{7_v5NjL9Sjn+(XPCrrgHUBj@YCRjV02H zkFTqHL;tsA5kE;j4B*9@N3PQ#GwA&BvvPJ6fFV?xhL34*>Juho@za^cRAl3tHR)>G%x~=LfJamcH3E6 zYu)jq~dv~iaMBUAN*gWI_hA=~`!ymSPi zoed?l>eSz^7LqU8iR2TM;a({skBWONq-^EDjA#O?1IxJv%w5RWIZl`VO4piHm7b|r z!%=RurWvfepH|Jk+bxMLEbHoN%@dE}Tw&FoC!6flQGRrYVX?%NlG@}8iPJ~oc;Ion zKChU9HdgX5N;QMoK-avl-cH?I>o)3^PuIS(&8Ryr*mlKRo7^u_NnD~Wi%WE@*xI*- z*60#nV4pV3=LeD1S~3>;aJMhwZ@}) zsc;aWR>_T@vA{+NPk! zlin&g>VN9nMSVeNn#Qz9z8F1JV-}SUJ{}csw>HS_%YFZ0t`FZH7BQ?YvTb|q8z7~u zs{GOp>mZ14YjCBQdp0SFrfiv^AWN6M`?RXB82^bVvAw7V-TzY$+QI!_b<_Wa^;?+Q zIQ$3F|35HFODTPJ8w_w^H=Z!WU6i5*d7^!NjD$^7@i) zO5Cz7xGlp=vh(s$pE6QX?a^lEv2i? zJp%|oTR&}F`qlGG#xoona8rDbeVCwjnqbc1q2K&*z2h)BuVl%588i{Yx*QRTnsk-9 z{6E;pdQR(J^G|LUx(ampBlYb27d}M+*q<}l<3%7V=DLHrgBh;#TEbS9wJEHOjjj^m z8F_k`xEAO*!P~ysdqIBaL_F1S^>SQ!MX9Xt5WdtjX4jpPdTxbKLj{c>dq$%CadG2h zP7YeKt6lf%-*#aEr-cuAi6ue;8nIcqql0S%U95Us0)joa;$TxOAV=`+g@rk56y+KE}Kw z2p^?BT_YFo0VvW}*-bnQ3TP<>x&4VEz4T){MBwI|cUpGYpWP<89vyrJO&>uZhM8p@PZ zjD9{6l1-FR%l}%EuSggqyx`y8-s(ja9C66qz=Ka31WE*L_v6CrBPi^C^gMd?9As5P zW#?JTtic$qUV0%w()?b_W6(LKDMbpSTrUb?N~W00q9KeSfj}Ku1KPWY8RQJEyFGnE z$cYggrZ!PNyY{)78Y3T)v6il{y`x32=a}Q}tnB2wZFd)40s>&9rtQN!K%|4n{Wdg_ z+lTRJDdEIyMV@2CNsLr-G@9;G|waiE1(on{DK)%pH1_#y2iN<~iC zr7f)mEM=1g?|^C~VXU%|(sPteUDR<(+EVTu?8q!pgf;oo(_JFr(xw45mPf^hc)Ytv z)Q1r=hZiWs?jxQ+o#Lc!U!+fJW0qWIp{F|^f{FFsl~x?AFTLr4Rd3PX+mhc@r6c%b z6#aTPn4{A%_3MPGB7-RLD{oHFiyeg=uC?yO0U{`(>tF|t`Yu{#e=vtE7m7O+FuJ1d z^inQ%5C=a-QZV0F(T|YN91{pN|4z@&IfzLIj=c(9oL_4dBYR>HFK%KYtM`9^lGcz_ z?11dHH9|&M<0&}g_kywnLu8wM{mhgBQ&Ns^1TQ)&Byum}!XjYr?t0v6HeN8Vz2N_# zenp<>&#_KA<^UU2QVM?}lC*&&6l2;YZ<;(KuzPf@i0SN9PLXq!(brH5Sm%JfP{5GJ z+-Ks-3q9u*CA0{z<`85mUcgkO&7c8H7f9^r-~IKTRYzby+M8-G+`Pf!7LoAr&<03Y zBgB2Vr_3!(jLS0G$=Ub2CWMl*7^Y zHB@J^HOe6?oT;E*FO&yKWK%`_$>G1Neza;)p5unqdMzJDfI4@4M&Dl;S_3gsj)*)j zEWDUgqts2GiOUV7*8(V3RW<2x8t=G6jC%ICAKT8^oLfbnETFGOV993x z!jT#Law0!{Dtc*HF6thxqc-x|Uv!tVg)Clw;iH$pqnC%) z4Nhuhkk3Oq;D+#9_-CI3(ldtBlC2PTdh~#eyZTAe@@vz%oFeW<{ws+bX ze%4d9ekGoEybCI{cqU1nY3koL3f1S0A1rOySJ~pszPxt^6N7(U)|M+2SI$un+`^bz zCN$}!!K=VYTRN$#s8R)dor%$9!s)YPyIC!>2`3UF&db{W>EjeB{RUEp?gE4LLa)1%vR(fYnD-TFu^h@*#&4WfP0 zlIPd*7p8mWhOTT?cl$%Rp_&RL%U(O)K6>HIusbeB`Ks3|h{2V5+Sh1*P6)HJgK(KD z+C95j4indwH2^ZRVVc3ptMS?l?$#~YM*cc-6CP@nwst{3yP*_QS2rIfz#BK-n3c~x z{utOUe?M49hGtvALajWaw&@i)tvm6z_F{I%?8jF2c_~^z-lBQN3&7!`c8G3@?2Lka z#7|c!tdu2(f#@MDPNQjQjC@Mr%6z;-*+(Zh_NJUId1sizRg;?gX6FPCE-3q4Uepi1 z!sv2LXnJ(l_v>tI)*C#PKXYVoxSvYVSM+gmmdO{qR$9Eeq{z5mC|*Z=Zx$n2vTUHt3wVnhG;?6$4p|I>fbklK#@!aohijJ{#NG?l7iX1}y3 z+3c`@goJlMsa3<~aXY!J(SfrX#QM+lnxuBe6`QQYZYtNw$K96uiSq$REFB9JC}$FU zM-aF-x5=6XPgKE60jqWSKfPp=rZD)EFKW`a)I^43x0P&VptIVdl-0m`iN-jthbrc5 zK|c`+Y_h6#fIGIL%Pb2-JlHg#-Cwz+_%F(5ii6G2*Yp8KqVKtg@UnePC_1YG*kepJ zGn|eC_4=D6u^23Tao=8PnJTyrWh^u%xFxyQn-Dl)@SV>x7Px~)FH^KmxzUBR-I zTcq?OkuCK-AFKg~?~-+x-?1@o+qNwN*T)usIuUQ>$qsoT~m|7{Qf_1 z=KtD13)u3Mw*U6e6YPJ#f1FMKpZ*88)OMmX1^#JlYB4XPh2j^eeL0|-rT{5IXMzi* zg)~VVlBlk&9oF-oFL}60mIydu&v;GF1fvJzl(1(`E z*y7Is`~CBL1o;`%Fd>&ED`?S;ITS*ulTMa2)o-HtOhzQ%a7;4^8}G_m9?76<&$~@) zcC~!vaZSy}`zvEaSb~RLsvlvEt3ev;rpE;aoz-gJes()c4>nB%x<8iZpqFtWGvO&5 z)+Z~`WNw!_>n6_Bt?SX|cboPGyLH&G5X|I(mCHog&7c|T(KpxQtwW&AN5vq_5^fzg z`w3bm(YywjJ%rk{3Sk@IdSCbBt@g(jlc$5;3_#c2v^EZFPDzOz_Z=9#>)qR*V;$u- z9aqFv-t{b)*F;h!LJHV;7}RGji=~oTH$EBli?O(Tqz_k#yKbvicwniH+MLSo-eSLD zY;t0eis~kY^5Yl45@jUYNjKdna^rt}V-$YT_BVAgmeZ7X$fdZV}XXJv^7B2 ze&`ypJG7pV>G!%i;Psxs|Dcuy%N>HA%_mRL{-Nai-s>T~3Y<;FQubhDA{jOZYSSgW zO8Mr7H8DB)nFHZ`4I|0f)0n2+`ofS*2H?0|IbJRt|_II_0P!9@GlzwckLY)8|VL*L{qG${a=a3 zZ>5%SF?=+wgiNpl1e}G&wUJeD*`vPE&;ll?B-Vxt_1`Ei+?$mf1q|_)tm1)BqbQ!w zY4>Zm0R~JQ0|cf5v^VhMLl*MV5ej~YzL_c~Y1{3Df7O*F{?vX!qyis1mwzfIYIUr< zOW<<#T8T0$IGBH`$Gx>PQW9g5)AZ{X>Lh3yG;U*V;@g!>6o9E0UF4$Z%ue_pxK0PGIaKAD(;>EsX6nlQ%}S?oDrt{% ztruLUyLJ=n&Ac$*?ua<&Pk94PvQoz>V^XkaS5R;I?itX9b^Qg`INqs`(GklW$+2R0 zyH2O9+Vu+YWMrs&l&in?lniDmxw7t;H_mcCNPalm+H!Q`YHKMoz8sjmfCs|W4{@{% zlx^;YPuQ8goWDP=hNj-aQ)?c`3{2^pUAGNH($Zn{+DTa_EGjk5OP*~8(_duQIlX5- zOH#gb--KhrU?6F9x=k?W<-BF2WR{!RG`$w9W1mUpSzFLsH(OMTo^@LB$z|i@xJ>?9 z_fhvZOd2fo0f$z%8N|{@md2q0WvQZ%V#mV8bJ6)ZxaQY$bLa-|i}@p|r&DxieoIf_>7ACCt`j5(w+k~b>@Cj*oPcKX`8N@i!Dc-Gg_0>> zNxd`L>*=z=7z6NDij+Sw)GS1XkHoj%e0`w~&@ zoVEY1SYfr>9Zz*8?ci|g&T;0XdydoK7**~kGTw9beVvsT)@DBZ^tNvBYXd``Z)nZKi2XoVA%Qrce9Gr)N14-?^ zqaq(?Cnw~;t-L;VBMqq*hP?CB)?7Fgr%nlB{F&?#&mVgpc=AdxVEB27|;9-LmG^xqUv{O@^F|GR{d7$4zvKGI^Zyz{ zVryz^@8qd(V`})Hf{*{w8hjI*}}rwU!pfft#av_FOD=?_wkTlKK?2W&)zA5FIFxG z_m-P`WiYqTHg4}`(zPa@IrAe#)ZHw_T$W)C`bN%1(+~acWID>uaB@&**?e0k3ZSHK ziN}}X7I&Dzlv`=ir`~;`yWWo0*@)4{oR(N{7$N9_mdBHpph@9%ZL=R;Pb)b#mvDxLl2RfJ5*Rw5Ou}1YOE3=T{({; zZz4>apH?yY79{IS-&Qz&xK&&(m+`5l_4#(v)q1>fz#ljDa4?sN-cY*wZPt(Jv1%53 z(~RWGm~(P;A6J66-bcEM0Oe{|AK9bb*cM%=wr#NT6b`pOBk901F2NAIz2~S^uKZ2P z3Q;PxRyV`&PAhUdm_O^dK)#CnE>)=e7$^RvPV3_2<@WjgNdJ8&{4=n--}q+6;K2?D zUeRKEeR&yk$67E+JgMRuT}=@_Vu2sUhD>1X%#5i&sO!i2cM6Ag(Zh-JqkHFUdeGb( z;fLv1t^%r~th!~1j$r1b#yA>*tvkh{qJ7Tzv>dxh!9%Iik2iJ3s1CIDvPU z-wy!k@D#um)4?MyuIR;>w+q%-Sj;z%+FUPvSK+kyD_QB-R&g?ZM}#FTx3M03n>-DC zP}rH_&7XNJ8LnTqWJB^dIVHn6@Pkzy?fxJwb8f(y;w@? z414TEfp=RPW%8V1frZkomPj5fhqHc(56ZZ|(G3hp{B-ig;XS+Gt-s%pM9ozmUi-5{ZWbhmVOBi%@ebSmB5DBWGsEhXLEA>6}PKhW>~ z^xng>pXdB>c;DG;*38~Bv(|d)6D8yX$Tr1`w&%dx=eR7)7Ul+5$6kr_uXQP-S_%?H zPRs(onLB49nq21noW^QeYl#WG8yXrgjO1|~#BD_iep`r>tcWysA1)DeNC|V+IsxaL zJo2r!A>ADNt86HtJ8}U|T;XZ58f>~Qr({_77xjL)LnMM=bOr=dt=d?;hLF7|2$$Cs zB+ehV$25ST=Gc#6(BUkKOuC+@H<>E6!;7n|amIZn1Zuy zqstW0@=NrT^HnoPYM^-DPosi*`FerrcO+p2sPnbq!$G;Hsy%Dy5~>p1ZK;HM*2zXb z{fVjKU5Lmm#Y@JI_1?xT_vk!KQ4C}L2uaXgAV~d*IxF$Zq$u^4HzobNY-0FCg%ONw zZ%eR|xGhesd4ug{yZHLbgtV@lwAh)I1c1Td#i{O4v1y!Mu&dv`G7V(zwG`%aO0aNi z_>wHsB<6Hscd%CBkM?XZPsg-W9(~zHjT+3+1_|D z4m1Qf-x$=LQsR)%6&u-!`@1=sdRP<_W_KDDX03Nnp9)8% zN7STZ*DEC0V~2$fz}c$Crz=z~TvXrDxfN#ie_E~O^H{gL?^ilnTU%L@$W;Jh6t9rw zS2fR|y(VuWk=!m#z)13-Lw?D7aynB zV#$BGG=9&vB2BFGt>%Jo-x(ZsuXseiyF{#5?JMw}iQZXcOzBvR))45lHQL>z>5JaV zfzpgth1o_5yCZWfJq$Rw=ExS?{a^#2G5taIe&lH0THeo(-hct4onE9zLaKg)x(N)rX+>f_{W-lxd zc7%22&ysUVtT$l=Zzm2U?!%oDY6^5vx>6o^yHX`|+k#`hs&P}Yvv@@8eo1_f0ZcPx zT&u_}tv^wX=_&fMtd}V-Qotilg#nv9c%%sAkSw zRBdoTcMjbYl^TUAPUyV0v_1`_M$Q6I1Da$Q8$aTjCzRYQ45hFWr(J6gI+pGvTL=}h zI|!DOLU#%X%PqGJR8EgP84Cd}M8;R@D1)0%SbiQXd*2L;xa-xjDlB^&-RP?trg-so zq`5Ow>}9S>BE+sdQClMG&hjbfnz6zMs`20zUbQ1H*k^sLP&Lpu6ydBe;kr=xq6(1n z0p^XtAK3s8F?&{NzZ9qqbb5;i6rUxuFJ0Bw<{eoaCsSC@tY5(Xt#YektA@zQw6n_s zcCEn6<=P=0$b_p=s2FAJ>RSI@6(4#vD6Hq218wCTy=#h&n#!kMpAZuozGE9lNKw$R zv0f--2bR;?X#FlcIQKQfSZlJ?+v}?WbKM~VYr0*b%EmbRQ7v&|v z$Q$$q`*Z!%qIQhHq-gC7rF6Cl7l8yY$NhY~gQODXWOf~=zis;gZR^P>RFcE6$>jyL z4ZT8b6T6Oo5ke~~Gi9Db82>h412BR|@&$OcRU{@Di+T!lFOB_m`Lq{FA*NU^u)|jh zyH;GcwPfba2ffV(%jT1acxrP~818!!_Osz#ccc&U!K*f)QVW#6#Ek3qeu_A?R>qvT zlsONi3t?**RC^AA;9ST?w$2~ON?J-)#fiydxTD9*?qVW%5MQX<>3d|0Cq4&DIyG)H zp@4TIwk>^+8geb=U%AQVHD9mAgPM6=m_jh8jdpO(;qR1>L1d%P5Nqs{-5DENMnKX-1t*of?aYM0_A~?aI?_FiWK)WfCdf@^mOF!^-g@Ujmq^w=exql_E3prwfgQs$fb?tyaI@%)D>|xQPe5Hv>_Gmx>P>959M6FQLoRpl@Hterv4>7 z`Kr~Rj|TaOTc_vb$_~xjUs(1wST|OB-IU5tc)PW|Obdnz5G~!RwGiaHyVWMpy-b(> zVt-!t0unFhCQ{>`uj61%IA~ET_J~SV&AGOT#DomDd^^~3w!enGW-596^u1N}J+td>oq71_kA$Ppy4_xH?Z zS>PYg2)c7O9T1LE6{qsr%*Dy#UX27RO=-uaZY}O!O8`>{2HY3NcP$@I!gwbEslDcq z8j@w^M8(qF8&?V(fHZ%ep$z&4?h6~^BEiS%i@nc!QDxgSW^68by@FEso;1fc;rixm z(bH<=az~x#3F5a_iWyqUP^GEfShze8EY9a7@%$R#70`}WOrc&_J8`Wb8Ql8spN>=%i zatt&tMxs$j*O9{ZzupK&l-S@EIUw00l|zlh0iENQRuVekAG_PZ$J00~g>vaP=TKQs z7_6~<6Cqn7>i4!SFN#@tyo9|46D?c0Yorc2zRyQ&sl~?BEoP`FsuC2PSX8!-3;m1G z*)UA*0MJa1ExZLzu}Y>;LcaW)sV*EX-j~LXtpr?dFA*e~%=GtdcG;gf5PN9VqYT~3 z(!7W6?0){a|69-pHINlG7E474bU0P$G!y|*^4d!K_h0KvqdjqPNvA^j0ztN*KDWNM z;78}JqpyNLl#$Kiw(T{VY|bvuw56d)Jk2S2tCv_N8vsLH@R9=GG3PcjG*4r0Adw!w zA3}&)E^?u?$d`G*Oh3&|L9XqUD3wJLDR&Sypd1T?Z}c+(pA$qdIAkboda!Q@#qcs} zLKPcVuYeOehG4lGHYdaGt|n7zod$~>d+lT8k8U$RVp%EMuEr>gC_~9$nOx8mj15Sa1l$WNZ4zqlTBcY9yOR;}w8ncp=g%DBl6b2V|CtT%4B;LoDK z1Y?59xk(yEa!;?nMJj;H;5uIznb)-J z3upu|DlqPfNU@E^uJF1Im}#1C^N1Ccr63Hc20s(Z>x9&)$wbOJRjbKDw4JS19gK}- z6{UJ?SyfP`1{V+LORDGNe!kjL3*7pHs2hFRYq-mpJcBeuXd4S@Y!2w!lWRlPqvh;Y z*9*?P7#R1HdCjj+H(XhlvNo(Bh1ryr(eao(5j5e=T#BvoUv!t0Y`s-Gl_GZ~h!JUfSi?DHyDWhK|ite+L z>)Rg6HZYwjpMt`3SGq{N3}?okbt^=}I;4 zQjVx zvEhcM>Nyvp{dWF0LCWmx7-a0tY}9nlomVCJ!co!N2Q;cx(GVHYuZ@h1jj_#9?U$Oe z>0a1XZQ#rBc*`c2<7=(#a)rko_GHf;UmN?LzjGX*Q`sD;%b+ytP4;y-gv?y@JINb~ zGn&=g!$1%MKH4h-?(#!VDIZO@h-0J=hQicN80!^e10DqV>OA5XiGRkBmyGuH!zwQk z*uD-9#D#&azi5#QD}CCY+=$e>CN#FjTu}M#sD`u{h=e4$=lk^A0~Wze)!02Se8&gT z{rF-KtD;f8A?>vWrrPr;Uk`ubky(o)>Y zfo!cKi0{s4KK+9G&8Y1y-Y(=I4Xx;!kp{@I(rS}A(;FUfG|wEdz{0~OP0lyz=iw+uu4Q0Ep{2R_8a(I#g>Za4z2E+uFTM5L8N^>kwfA70sMC@6A9pX|Gz~ zH$uDlzAn=Nn=Mgxq(}>N5|JV!+y0F-Hns&h8_q{W>BO=72>-Tti9m2OQldMfGpDsd zN;GvAWqmugt*O<$6mCLZ@0pa_%1iKhB zbkXmX($Yhsf^=0PT`bbmL0u=p)DQ%8Qzq0Z<=4w5^9v8k8K`eFJ&1eE(@NyHiBi|@ z3T&0%!9WTY?@aHUCG|o7Ooib4PM1||gJZ`~xCb!lu);|y zuqM~&C-~>{sG3EdVbo_jF_Mvm9|%^zqDY;GUjVbI5WuZ|P&xSH+4p;6-ZEUu959Cl z9NeJ^P+$azb<@*<33FiXvYH$SYp!(|-nBR=w{3criawuC6ToOJ#rGi0aHW!rMTj77 zIHsUgK{%XdrO8e)&GVo(pH|4{^|Hl8gZU;76VaWU$u-I`BfZHI`b|(eyRF{D`cqbn zla6Ip%cQ_8h#F3hh`DX?%ViA9dwEwTmz!2DdcFAs$!-Vr;P81PySqxPzW5dXgT9uF z(`?iI-kP27(4^3O!dGl`-K90}|O;iK34OE*jl|Hr4;HpR037|wD zF4j*~xBjrt8&hL|=*n*$^zFTtvIc~}Hg2exY<2Ey=z}|A!mOg&s|OZSmhG}Wd%H~u z(49M`tyR7Mo`*1Nw@d1{UBb<}NmMD;BogGBW%qVadPufo!?vQ~yDyxU@LUL;#YpEPy4xmjh%TWLc!!^f@Q6v8(319&CPNmNIw!QzOud);nW@RA%T9lhrw>i$tA0T?D!W~)25(`vQzX-O!AYvjk6BdGEckLG$a2BoTgHE$x!|`3%bFjuP5v?rq>}43r~hte z&?ya%x(>SZut7=H;pwTTU`6(H5Vim{YxF1D+!UtoTE0@n+a?IiRCEwcUH}pB+_Qt~atY4#`>Q(insX zgcxd7=WS5kA>L_?yc0NE9H)F0AGi)i3=U9~#qzO2WI);U__!LC_u=y z*qL!bd7v07@L@CYRgs0TMQ-Eat#s?5VzyucA0wDKp>VJ3VQi#0K6G^zy0!VWwG=0K zH|!;Y%Q4G4UU5@=!5LCc(>l!MT?p|7;Zejr6O;izQ`S4wV{n`AZx z$!Nz?l-hbd0fsHOVwaJ^72z3u1;|~cI%5k640DnxKcGDowB4!z3M9S5q)!I-Y_7Rz z-=#f(%xc6aNU4C=bme6;u?kglKoxP(&b|=JRuQ6{`24Yx4o3L}(>KMP;mCro#adx_ z{Z1;=TE$AK^se>hCHYgw_l)z;xbH6H!61)n8GwBcT>}bVT5Ex7Kn1KYcd+R)vX@$B zbl!tW_3ORb7MaFeZ{`|(0YrnxA`aR8A=8h9WZ4YTpeC`cp6Iz$AmN(Y-J^1!@qiNf zPN6hkSpeZ;{BS@(SWh3{ZwxJr9;OmQfS!z@rII2v5NHOVF8;&yLv8%VxEc6D(Wr+& z`o%FaN5*U}IV}N`YYw9`^DDy&1x8BpPyZwIr!suS&Ds|Y}OCyPrYQ6KUrPDDM@p4%R zWhE8HAl&0T(d*e_Yvetm)IzAGHYy?ixTiv0#{GBVcMH}TgviZDxcA{Wttbtmu(Tq< zZLw)YqLBws@)vutd1i9U^o)7UI z<^q90bkh4Je0D{WTCOO}AYIbwa1tS?oLX;(JZjKFQ}KwfFEaDp@Cyk;pM2NlA>i&R zx>0>ItC-Xkp{2qSIOE8mElhM->L8I6PQ?p_i92=SiYp+BR)g$PtqgJgwij)i9oIhg zD|C}B_t*7{BF%ob*Y?N0HL+Gv-rdI2?Ly>tNqJ-t(ffr(I-8gwBikT~bCxSd!tl_pneEWtc}vYq_=roYA}jOt zmB?QlODh?^W`;m8~5ctB-lB7uOW+u|46zJ+TX#9H|=}zbfxMs#U*g zD>IWQcb&Bra`PGbr7}&#Way#$>2u}9A$_1Mx=_neU^ZM)p~0h z5EN*-AaNI!d9qkGi1!d;8H2k-@fKj98lK&yqQP2HJdU!U>g)xY5?cW`uhf$7E^A)+ zWkpnb?5!O*pMW4@?z6nbtHq2qAVQ9tw|v+2R+zKb8{K>y_4Cr1`{I;FZ?Vg~(`fy3 z)5NOTL3EFPY{v4V@~BnnY#5cBxZ0h@-dnwd!<%Yg+by&g(m^d2tGRb@FQ{o7F9^#P zP{@+IeJO}t_)}dR4+lSeiC0KJSaiYSV1nO6=*V~D`_|Y^*mDDqA6umx=Qv*mx^Pg# z#(kND$BOKskLjJ0h`)(m%j{8xf4sm4t|yS3yg7;&<@FR;t9xg_S1Bg)?u~}B%n_p*WSTaxTPTueJMu3U z@$qws7$bugnM{WAL0i5Oi<^2IgCbYeDJdLZ*i+Ft0=I-1BQEgBjgqx1PDSUK)$%@ZO~N`m#unzLoNT zN!low6rK}u@a>W#K>~VmngR{U>gTEa#qU!zC~sE85y`!V?0f~ohs09s+b_9s+1!gu zz+vM=m7ydM#`dhr2Z5NZnP?Q8^Wa5pp%}Bbt(<}WyxVtyg8^EUsde#_{ znxI|7>1#1+8t#?|pjMR88EzKnI25|1j%6Jo}CLM*8dK#Zb6#=D2a z)_V93S$)&db zE3Ho=C}pyCUb`XVsGLi5Yvnjw`6PMS(C%>E*|Wo~;v`h>mc7N*5-h*ZO9_(|<#V-? zb!oA0T6G3C({C`kWi7ukY=qJ)4T@xMn5xUek@V3P1zmgUdIo-K)e$fX47nTc(-wHxb3An8NNhl-T`yxezG2(nym7jRj|}lEoJ3C zKS&#yi=nQ`PT{iD`0n(5I;Ft+36IGl*)%GA%zG1S@j2+0)xLEFHin>iS$NJdI4BM4 z8Xc6r*HCyO$kjZ4qZbWXubOU&DZQZX{rkOE;OmamXI$e=M{h zzU&3FHP1!mC{!$%46$ii8IbHbTp7q~)xB)horFMF%C;O|wwl!8@H67@0OuHFWXiI` z0N1h2#PEt$9rSL-PYxZkC*|pX<^tq!r@OT%^)faQE{cZI=F^J9P&LCh& z@sIze96hyol8~FrtE_OEJTqLIbwIL9J;Zw@~z%csf z1|5nw*r0lReW-*HDm=bzZ(p!sv}&H|R7Ne;TE0ZZK_mEHiL`)wuQ`Zd4Tp&3@>MUT zcSv-@lrdw^t|49XsG}e;HF#)9!ixh*9nY?Wg*7F|s9XI8{QE1>Xac~|my|DslRod~ z2&xrJI{qu4kDe#zyxJ_ItPK~Jyp`iGtw)>}iU+<92h232-+HTxlTcQSN=|xo>(OoLj(#_sRF>ULnZhVy-Ja^%*QgLL$rf%zQ=*XDE=rHo zfe2rEVnnp#=;*{-D37sC*R9PQvvk#$eoT-_KEi#C#IzD-bXOE4CQ-|(8sz!5>A1kx z+muXCD$md+q1`pO&(&EL1U4Glb|qY^mZ5rPp0rGH5%sxrdRjDtNCq)PC%8;P;6c2a zw%k(aWnzD?LrDOoMiZAb9O6Zfgf@~Qs{R)n{}uk5%bk^X{adFm8WBEJxw>7-I`j)d zr+UD|F?p=ZTawRoIpvU8mE!lZJ-5ZlRKmF+LPfJ#Hw6^?@HEhi=LqZoi)cii`k?bl zfd$vMENyDELw3E9Xh(TB`%?qk*@Bj@T4VGx6a~+kCbVRg847P7ao$o{|1sWVaJU@q zg`N?xD?+P-#t~*x)O)Ak`|!|R?WZd`Ph{4nD!`tXGazj9hZXPtVKLXJ+zZHOL~Pul zffy8WL?fy2pnP^2K@LA`zI=)d20|B#l2efvp{Sm@+5BN#oKCi=CLcaPXz-Zx08Yf% zhK~y}6`a*ot?m2z`t2vJa>t~-))U7dNlsV%VUI60exw5YnIRLW-2A>|k-m&4}1+*9P*R*mO5+bqH%);~Ob8rvS@OnC<93fLX z2#yd?Wd2CdBM;P!IT|@EQS3ytGWM_3L3}(S1X~EJCCL-Ra`w_vdjKaK~d5`p<<`*IP;jYkI#oXdEQ z252Ch4xK`gu*mBQQ=*U9?R4!J+iFT>r*v_$nuJ6|90gh~$SHQGlImj=m7(0-0sBZo z0nFUHG0ykQmPLxm>9hOqg(A_7RlKsibDMPK(s8}~wk*DmcibO2;Ca9`l5d;AC}-m7 zXYx12iz}ZsIjPyP;*-qA9V)XEDb}5tXDC+K5O(yi%c2H#I&0(}C%`lZN=5I*v5tk{ zq@=}DEd>Q5a^!VU6 z0eT2jyCZ&ojU=@mGw1so=xNLbUn6loe{qang#Po;!@(3vKuz`p0^*P8nb&DseXr*U zuX6)=oxFMG@Lg{ISdRb>=>Q*U)irTD1X+Cy!1sp>;qUc`qak49(Aw(nO>c~V9yFD%jd=4P}5q0u!)CbWr8*4lJ-$Z>dtO+hj{U$&*)Kd!x2=z%@7XT@+f84gd zg(*Nj^o^k%z>a?-|H29n$kZ`5y_kUQs3^cP68;G?D&F77)|NK9|FrfOm=9r{3fGx_ zV*t!iL?9rfCz$qxe_{TkEc|ycBskn2Y%_`d?M?|3?0WIUeflZys)rGXPA%#PXv{ z9K8Jt*~ZS=-df+<;&0T4GW)-9G{IT|%SwQwK>=i&5uRAXB>FGZzn(sSgZ{53{`cVp z`xod#qxblY;{O7DICX#Xi$V4$;Dalmd0&D*47rC3{Yfhep920rhFXbrujf7>UBV3E zM}I&_c+yxsLo0m~OI@UcfHr9a{Lb8u__Il7;){KEep`rxU+_hT0I$|o%U z49X`A|Et^l$5v03jUO{~)I4GMN$2?2c6zvjr&6bn@lhI|;QulHb)O2K{+i-vwaLd6 zC(S=o{BwZ+PYeoxhnjVh{6o-B$_(|adK=KHp`)6c7KBxRWqNi~rkBKx;e@*mfRLSo#J&m+@ z%#?-pUzq+i?&9|tp9Ts%W>mrWFO2^lHt>6tPY;qGQx;(TO!7a#r)>^W^E)!(;d)%BS!@o+|&h=Ktq?@acl?F@6rs hU-*xgcL30TeIEe_EK-1g$N+yGfcJL@+K0FH{{dOjytV)U literal 0 HcmV?d00001 diff --git a/python/setup.py b/python/setup.py index 2644d3e79dea..cfc83c68e3df 100644 --- a/python/setup.py +++ b/python/setup.py @@ -194,7 +194,7 @@ def _supports_symlinks(): 'pyspark.examples.src.main.python': ['*.py', '*/*.py']}, scripts=scripts, license='http://www.apache.org/licenses/LICENSE-2.0', - install_requires=['py4j==0.10.4'], + install_requires=['py4j==0.10.6'], setup_requires=['pypandoc'], extras_require={ 'ml': ['numpy>=1.7'], diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index e5131e636dc0..1dd071591804 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -1124,7 +1124,7 @@ private[spark] class Client( val pyArchivesFile = new File(pyLibPath, "pyspark.zip") require(pyArchivesFile.exists(), s"$pyArchivesFile not found; cannot run pyspark application in YARN mode.") - val py4jFile = new File(pyLibPath, "py4j-0.10.4-src.zip") + val py4jFile = new File(pyLibPath, "py4j-0.10.6-src.zip") require(py4jFile.exists(), s"$py4jFile not found; cannot run pyspark application in YARN mode.") Seq(pyArchivesFile.getAbsolutePath(), py4jFile.getAbsolutePath()) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 59adb7e22d18..fc78bc488b11 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -249,7 +249,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { // needed locations. val sparkHome = sys.props("spark.test.home") val pythonPath = Seq( - s"$sparkHome/python/lib/py4j-0.10.4-src.zip", + s"$sparkHome/python/lib/py4j-0.10.6-src.zip", s"$sparkHome/python") val extraEnvVars = Map( "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator), diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index f2d9e6b568a9..bac154e10ae6 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -28,6 +28,6 @@ export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"${SPARK_HOME}/conf"}" # Add the PySpark classes to the PYTHONPATH: if [ -z "${PYSPARK_PYTHONPATH_SET}" ]; then export PYTHONPATH="${SPARK_HOME}/python:${PYTHONPATH}" - export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.4-src.zip:${PYTHONPATH}" + export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.6-src.zip:${PYTHONPATH}" export PYSPARK_PYTHONPATH_SET=1 fi From ab866f117378e64dba483ead51b769ae7be31d4d Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 5 Jul 2017 18:26:28 -0700 Subject: [PATCH 047/125] [SPARK-21248][SS] The clean up codes in StreamExecution should not be interrupted ## What changes were proposed in this pull request? This PR uses `runUninterruptibly` to avoid that the clean up codes in StreamExecution is interrupted. It also removes an optimization in `runUninterruptibly` to make sure this method never throw `InterruptedException`. ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #18461 from zsxwing/SPARK-21248. --- .../org/apache/spark/util/UninterruptibleThread.scala | 10 +--------- .../apache/spark/util/UninterruptibleThreadSuite.scala | 5 ++--- .../sql/execution/streaming/StreamExecution.scala | 6 +++++- 3 files changed, 8 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala index 27922b31949b..6a58ec142dd7 100644 --- a/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala +++ b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala @@ -55,9 +55,6 @@ private[spark] class UninterruptibleThread( * Run `f` uninterruptibly in `this` thread. The thread won't be interrupted before returning * from `f`. * - * If this method finds that `interrupt` is called before calling `f` and it's not inside another - * `runUninterruptibly`, it will throw `InterruptedException`. - * * Note: this method should be called only in `this` thread. */ def runUninterruptibly[T](f: => T): T = { @@ -73,12 +70,7 @@ private[spark] class UninterruptibleThread( uninterruptibleLock.synchronized { // Clear the interrupted status if it's set. - if (Thread.interrupted() || shouldInterruptThread) { - shouldInterruptThread = false - // Since it's interrupted, we don't need to run `f` which may be a long computation. - // Throw InterruptedException as we don't have a T to return. - throw new InterruptedException() - } + shouldInterruptThread = Thread.interrupted() || shouldInterruptThread uninterruptible = true } try { diff --git a/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadSuite.scala b/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadSuite.scala index 39b31f8ddeab..6a190f63ac9d 100644 --- a/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadSuite.scala @@ -68,7 +68,6 @@ class UninterruptibleThreadSuite extends SparkFunSuite { Uninterruptibles.awaitUninterruptibly(interruptLatch, 10, TimeUnit.SECONDS) try { runUninterruptibly { - assert(false, "Should not reach here") } } catch { case _: InterruptedException => hasInterruptedException = true @@ -80,8 +79,8 @@ class UninterruptibleThreadSuite extends SparkFunSuite { t.interrupt() interruptLatch.countDown() t.join() - assert(hasInterruptedException === true) - assert(interruptStatusBeforeExit === false) + assert(hasInterruptedException === false) + assert(interruptStatusBeforeExit === true) } test("nested runUninterruptibly") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index d5f8d2acba92..10c42a7338e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -357,7 +357,11 @@ class StreamExecution( if (!NonFatal(e)) { throw e } - } finally { + } finally microBatchThread.runUninterruptibly { + // The whole `finally` block must run inside `runUninterruptibly` to avoid being interrupted + // when a query is stopped by the user. We need to make sure the following codes finish + // otherwise it may throw `InterruptedException` to `UncaughtExceptionHandler` (SPARK-21248). + // Release latches to unblock the user codes since exception can happen in any place and we // may not get a chance to release them startLatch.countDown() From 75b168fd30bb9a52ae223b6f1df73da4b1316f2e Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 6 Jul 2017 14:18:50 +0800 Subject: [PATCH 048/125] [SPARK-21308][SQL] Remove SQLConf parameters from the optimizer ### What changes were proposed in this pull request? This PR removes SQLConf parameters from the optimizer rules ### How was this patch tested? The existing test cases Author: gatorsmile Closes #18533 from gatorsmile/rmSQLConfOptimizer. --- .../optimizer/CostBasedJoinReorder.scala | 7 ++-- .../sql/catalyst/optimizer/Optimizer.scala | 36 +++++++++---------- .../optimizer/StarSchemaDetection.scala | 4 ++- .../sql/catalyst/optimizer/expressions.scala | 14 ++++---- .../spark/sql/catalyst/optimizer/joins.scala | 6 ++-- .../BinaryComparisonSimplificationSuite.scala | 2 +- .../BooleanSimplificationSuite.scala | 2 +- .../optimizer/CombiningLimitsSuite.scala | 2 +- .../optimizer/ConstantFoldingSuite.scala | 2 +- .../optimizer/DecimalAggregatesSuite.scala | 2 +- .../optimizer/EliminateMapObjectsSuite.scala | 2 +- .../optimizer/JoinOptimizationSuite.scala | 2 +- .../catalyst/optimizer/JoinReorderSuite.scala | 27 +++++++++++--- .../optimizer/LimitPushdownSuite.scala | 2 +- .../optimizer/OptimizeCodegenSuite.scala | 2 +- .../catalyst/optimizer/OptimizeInSuite.scala | 24 +++++++------ .../StarJoinCostBasedReorderSuite.scala | 36 ++++++++++++++----- .../optimizer/StarJoinReorderSuite.scala | 25 ++++++++++--- .../optimizer/complexTypesSuite.scala | 2 +- .../spark/sql/catalyst/plans/PlanTest.scala | 4 +-- .../execution/OptimizeMetadataOnlyQuery.scala | 8 ++--- .../spark/sql/execution/SparkOptimizer.scala | 6 ++-- .../internal/BaseSessionStateBuilder.scala | 2 +- 23 files changed, 137 insertions(+), 82 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala index 3a7543e2141e..db7baf6e9bc7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala @@ -32,7 +32,10 @@ import org.apache.spark.sql.internal.SQLConf * We may have several join reorder algorithms in the future. This class is the entry of these * algorithms, and chooses which one to use. */ -case class CostBasedJoinReorder(conf: SQLConf) extends Rule[LogicalPlan] with PredicateHelper { +object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper { + + private def conf = SQLConf.get + def apply(plan: LogicalPlan): LogicalPlan = { if (!conf.cboEnabled || !conf.joinReorderEnabled) { plan @@ -379,7 +382,7 @@ object JoinReorderDPFilters extends PredicateHelper { if (conf.joinReorderDPStarFilter) { // Compute the tables in a star-schema relationship. - val starJoin = StarSchemaDetection(conf).findStarJoins(items, conditions.toSeq) + val starJoin = StarSchemaDetection.findStarJoins(items, conditions.toSeq) val nonStarJoin = items.filterNot(starJoin.contains(_)) if (starJoin.nonEmpty && nonStarJoin.nonEmpty) { 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 946fa7bae019..d82af94dbffb 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 @@ -34,10 +34,10 @@ import org.apache.spark.sql.types._ * Abstract class all optimizers should inherit of, contains the standard batches (extending * Optimizers can override this. */ -abstract class Optimizer(sessionCatalog: SessionCatalog, conf: SQLConf) +abstract class Optimizer(sessionCatalog: SessionCatalog) extends RuleExecutor[LogicalPlan] { - protected val fixedPoint = FixedPoint(conf.optimizerMaxIterations) + protected def fixedPoint = FixedPoint(SQLConf.get.optimizerMaxIterations) def batches: Seq[Batch] = { Batch("Eliminate Distinct", Once, EliminateDistinct) :: @@ -77,11 +77,11 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: SQLConf) Batch("Operator Optimizations", fixedPoint, Seq( // Operator push down PushProjectionThroughUnion, - ReorderJoin(conf), + ReorderJoin, EliminateOuterJoin, PushPredicateThroughJoin, PushDownPredicate, - LimitPushDown(conf), + LimitPushDown, ColumnPruning, InferFiltersFromConstraints, // Operator combine @@ -92,10 +92,10 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: SQLConf) CombineLimits, CombineUnions, // Constant folding and strength reduction - NullPropagation(conf), + NullPropagation, ConstantPropagation, FoldablePropagation, - OptimizeIn(conf), + OptimizeIn, ConstantFolding, ReorderAssociativeOperator, LikeSimplification, @@ -117,11 +117,11 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: SQLConf) CombineConcats) ++ extendedOperatorOptimizationRules: _*) :: Batch("Check Cartesian Products", Once, - CheckCartesianProducts(conf)) :: + CheckCartesianProducts) :: Batch("Join Reorder", Once, - CostBasedJoinReorder(conf)) :: + CostBasedJoinReorder) :: Batch("Decimal Optimizations", fixedPoint, - DecimalAggregates(conf)) :: + DecimalAggregates) :: Batch("Object Expressions Optimization", fixedPoint, EliminateMapObjects, CombineTypedFilters) :: @@ -129,7 +129,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: SQLConf) ConvertToLocalRelation, PropagateEmptyRelation) :: Batch("OptimizeCodegen", Once, - OptimizeCodegen(conf)) :: + OptimizeCodegen) :: Batch("RewriteSubquery", Once, RewritePredicateSubquery, CollapseProject) :: Nil @@ -178,8 +178,7 @@ class SimpleTestOptimizer extends Optimizer( new SessionCatalog( new InMemoryCatalog, EmptyFunctionRegistry, - new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true)), - new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true)) + new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true))) /** * Remove redundant aliases from a query plan. A redundant alias is an alias that does not change @@ -288,7 +287,7 @@ object RemoveRedundantProject extends Rule[LogicalPlan] { /** * Pushes down [[LocalLimit]] beneath UNION ALL and beneath the streamed inputs of outer joins. */ -case class LimitPushDown(conf: SQLConf) extends Rule[LogicalPlan] { +object LimitPushDown extends Rule[LogicalPlan] { private def stripGlobalLimitIfPresent(plan: LogicalPlan): LogicalPlan = { plan match { @@ -1077,8 +1076,7 @@ object CombineLimits extends Rule[LogicalPlan] { * the join between R and S is not a cartesian product and therefore should be allowed. * The predicate R.r = S.s is not recognized as a join condition until the ReorderJoin rule. */ -case class CheckCartesianProducts(conf: SQLConf) - extends Rule[LogicalPlan] with PredicateHelper { +object CheckCartesianProducts extends Rule[LogicalPlan] with PredicateHelper { /** * Check if a join is a cartesian product. Returns true if * there are no join conditions involving references from both left and right. @@ -1090,7 +1088,7 @@ case class CheckCartesianProducts(conf: SQLConf) } def apply(plan: LogicalPlan): LogicalPlan = - if (conf.crossJoinEnabled) { + if (SQLConf.get.crossJoinEnabled) { plan } else plan transform { case j @ Join(left, right, Inner | LeftOuter | RightOuter | FullOuter, condition) @@ -1112,7 +1110,7 @@ case class CheckCartesianProducts(conf: SQLConf) * This uses the same rules for increasing the precision and scale of the output as * [[org.apache.spark.sql.catalyst.analysis.DecimalPrecision]]. */ -case class DecimalAggregates(conf: SQLConf) extends Rule[LogicalPlan] { +object DecimalAggregates extends Rule[LogicalPlan] { import Decimal.MAX_LONG_DIGITS /** Maximum number of decimal digits representable precisely in a Double */ @@ -1130,7 +1128,7 @@ case class DecimalAggregates(conf: SQLConf) extends Rule[LogicalPlan] { we.copy(windowFunction = ae.copy(aggregateFunction = Average(UnscaledValue(e)))) Cast( Divide(newAggExpr, Literal.create(math.pow(10.0, scale), DoubleType)), - DecimalType(prec + 4, scale + 4), Option(conf.sessionLocalTimeZone)) + DecimalType(prec + 4, scale + 4), Option(SQLConf.get.sessionLocalTimeZone)) case _ => we } @@ -1142,7 +1140,7 @@ case class DecimalAggregates(conf: SQLConf) extends Rule[LogicalPlan] { val newAggExpr = ae.copy(aggregateFunction = Average(UnscaledValue(e))) Cast( Divide(newAggExpr, Literal.create(math.pow(10.0, scale), DoubleType)), - DecimalType(prec + 4, scale + 4), Option(conf.sessionLocalTimeZone)) + DecimalType(prec + 4, scale + 4), Option(SQLConf.get.sessionLocalTimeZone)) case _ => ae } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/StarSchemaDetection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/StarSchemaDetection.scala index ca729127e7d1..1f20b7661489 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/StarSchemaDetection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/StarSchemaDetection.scala @@ -28,7 +28,9 @@ import org.apache.spark.sql.internal.SQLConf /** * Encapsulates star-schema detection logic. */ -case class StarSchemaDetection(conf: SQLConf) extends PredicateHelper { +object StarSchemaDetection extends PredicateHelper { + + private def conf = SQLConf.get /** * Star schema consists of one or more fact tables referencing a number of dimension diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 66b8ca62e5e4..6c83f4790004 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -173,12 +173,12 @@ object ReorderAssociativeOperator extends Rule[LogicalPlan] { * 2. Replaces [[In (value, seq[Literal])]] with optimized version * [[InSet (value, HashSet[Literal])]] which is much faster. */ -case class OptimizeIn(conf: SQLConf) extends Rule[LogicalPlan] { +object OptimizeIn extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsDown { case expr @ In(v, list) if expr.inSetConvertible => val newList = ExpressionSet(list).toSeq - if (newList.size > conf.optimizerInSetConversionThreshold) { + if (newList.size > SQLConf.get.optimizerInSetConversionThreshold) { val hSet = newList.map(e => e.eval(EmptyRow)) InSet(v, HashSet() ++ hSet) } else if (newList.size < list.size) { @@ -414,7 +414,7 @@ object LikeSimplification extends Rule[LogicalPlan] { * equivalent [[Literal]] values. This rule is more specific with * Null value propagation from bottom to top of the expression tree. */ -case class NullPropagation(conf: SQLConf) extends Rule[LogicalPlan] { +object NullPropagation extends Rule[LogicalPlan] { private def isNullLiteral(e: Expression): Boolean = e match { case Literal(null, _) => true case _ => false @@ -423,9 +423,9 @@ case class NullPropagation(conf: SQLConf) extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsUp { case e @ WindowExpression(Cast(Literal(0L, _), _, _), _) => - Cast(Literal(0L), e.dataType, Option(conf.sessionLocalTimeZone)) + Cast(Literal(0L), e.dataType, Option(SQLConf.get.sessionLocalTimeZone)) case e @ AggregateExpression(Count(exprs), _, _, _) if exprs.forall(isNullLiteral) => - Cast(Literal(0L), e.dataType, Option(conf.sessionLocalTimeZone)) + Cast(Literal(0L), e.dataType, Option(SQLConf.get.sessionLocalTimeZone)) case ae @ AggregateExpression(Count(exprs), _, false, _) if !exprs.exists(_.nullable) => // This rule should be only triggered when isDistinct field is false. ae.copy(aggregateFunction = Count(Literal(1))) @@ -552,14 +552,14 @@ object FoldablePropagation extends Rule[LogicalPlan] { /** * Optimizes expressions by replacing according to CodeGen configuration. */ -case class OptimizeCodegen(conf: SQLConf) extends Rule[LogicalPlan] { +object OptimizeCodegen extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case e: CaseWhen if canCodegen(e) => e.toCodegen() } private def canCodegen(e: CaseWhen): Boolean = { val numBranches = e.branches.size + e.elseValue.size - numBranches <= conf.maxCaseBranchesForCodegen + numBranches <= SQLConf.get.maxCaseBranchesForCodegen } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index bb97e2c808b9..edbeaf273fd6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.internal.SQLConf * * If star schema detection is enabled, reorder the star join plans based on heuristics. */ -case class ReorderJoin(conf: SQLConf) extends Rule[LogicalPlan] with PredicateHelper { +object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { /** * Join a list of plans together and push down the conditions into them. * @@ -87,8 +87,8 @@ case class ReorderJoin(conf: SQLConf) extends Rule[LogicalPlan] with PredicateHe def apply(plan: LogicalPlan): LogicalPlan = plan transform { case ExtractFiltersAndInnerJoins(input, conditions) if input.size > 2 && conditions.nonEmpty => - if (conf.starSchemaDetection && !conf.cboEnabled) { - val starJoinPlan = StarSchemaDetection(conf).reorderStarJoins(input, conditions) + if (SQLConf.get.starSchemaDetection && !SQLConf.get.cboEnabled) { + val starJoinPlan = StarSchemaDetection.reorderStarJoins(input, conditions) if (starJoinPlan.nonEmpty) { val rest = input.filterNot(starJoinPlan.contains(_)) createOrderedJoin(starJoinPlan ++ rest, conditions) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala index 2a04bd588dc1..a313681eeb8f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala @@ -33,7 +33,7 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper Batch("AnalysisNodes", Once, EliminateSubqueryAliases) :: Batch("Constant Folding", FixedPoint(50), - NullPropagation(conf), + NullPropagation, ConstantFolding, BooleanSimplification, SimplifyBinaryComparison, 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 c6345b60b744..56399f4831a6 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 @@ -35,7 +35,7 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { Batch("AnalysisNodes", Once, EliminateSubqueryAliases) :: Batch("Constant Folding", FixedPoint(50), - NullPropagation(conf), + NullPropagation, ConstantFolding, BooleanSimplification, PruneFilters) :: Nil 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 ac71887c16f9..87ad81db11b6 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 @@ -32,7 +32,7 @@ class CombiningLimitsSuite extends PlanTest { Batch("Combine Limit", FixedPoint(10), CombineLimits) :: Batch("Constant Folding", FixedPoint(10), - NullPropagation(conf), + NullPropagation, ConstantFolding, BooleanSimplification, SimplifyConditionals) :: Nil diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index 25c592b9c1dd..641c89873dcc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -33,7 +33,7 @@ class ConstantFoldingSuite extends PlanTest { Batch("AnalysisNodes", Once, EliminateSubqueryAliases) :: Batch("ConstantFolding", Once, - OptimizeIn(conf), + OptimizeIn, ConstantFolding, BooleanSimplification) :: Nil } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala index cc4fb3a244a9..711294ed6192 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala @@ -29,7 +29,7 @@ class DecimalAggregatesSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Decimal Optimizations", FixedPoint(100), - DecimalAggregates(conf)) :: Nil + DecimalAggregates) :: Nil } val testRelation = LocalRelation('a.decimal(2, 1), 'b.decimal(12, 1)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateMapObjectsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateMapObjectsSuite.scala index d4f37e2a5e87..157472c2293f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateMapObjectsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateMapObjectsSuite.scala @@ -31,7 +31,7 @@ class EliminateMapObjectsSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = { Batch("EliminateMapObjects", FixedPoint(50), - NullPropagation(conf), + NullPropagation, SimplifyCasts, EliminateMapObjects) :: Nil } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala index a6584aa5fbba..2f30a78f0321 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala @@ -37,7 +37,7 @@ class JoinOptimizationSuite extends PlanTest { CombineFilters, PushDownPredicate, BooleanSimplification, - ReorderJoin(conf), + ReorderJoin, PushPredicateThroughJoin, ColumnPruning, CollapseProject) :: Nil diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala index 71db4e2e0ec4..2fb587d50a4c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala @@ -24,25 +24,42 @@ import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.statsEstimation.{StatsEstimationTestBase, StatsTestPlan} -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.{CBO_ENABLED, JOIN_REORDER_ENABLED} class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { - override val conf = new SQLConf().copy(CBO_ENABLED -> true, JOIN_REORDER_ENABLED -> true) - object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Operator Optimizations", FixedPoint(100), CombineFilters, PushDownPredicate, - ReorderJoin(conf), + ReorderJoin, PushPredicateThroughJoin, ColumnPruning, CollapseProject) :: Batch("Join Reorder", Once, - CostBasedJoinReorder(conf)) :: Nil + CostBasedJoinReorder) :: Nil + } + + var originalConfCBOEnabled = false + var originalConfJoinReorderEnabled = false + + override def beforeAll(): Unit = { + super.beforeAll() + originalConfCBOEnabled = conf.cboEnabled + originalConfJoinReorderEnabled = conf.joinReorderEnabled + conf.setConf(CBO_ENABLED, true) + conf.setConf(JOIN_REORDER_ENABLED, true) + } + + override def afterAll(): Unit = { + try { + conf.setConf(CBO_ENABLED, originalConfCBOEnabled) + conf.setConf(JOIN_REORDER_ENABLED, originalConfJoinReorderEnabled) + } finally { + super.afterAll() + } } /** Set up tables and columns for testing */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala index d8302dfc9462..f50e2e86516f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala @@ -32,7 +32,7 @@ class LimitPushdownSuite extends PlanTest { Batch("Subqueries", Once, EliminateSubqueryAliases) :: Batch("Limit pushdown", FixedPoint(100), - LimitPushDown(conf), + LimitPushDown, CombineLimits, ConstantFolding, BooleanSimplification) :: Nil diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCodegenSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCodegenSuite.scala index 9dc6738ba04b..b71067c0af3a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCodegenSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCodegenSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.rules._ class OptimizeCodegenSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { - val batches = Batch("OptimizeCodegen", Once, OptimizeCodegen(conf)) :: Nil + val batches = Batch("OptimizeCodegen", Once, OptimizeCodegen) :: Nil } protected def assertEquivalent(e1: Expression, e2: Expression): Unit = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala index d8937321ecb9..6a77580b29a2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala @@ -34,10 +34,10 @@ class OptimizeInSuite extends PlanTest { Batch("AnalysisNodes", Once, EliminateSubqueryAliases) :: Batch("ConstantFolding", FixedPoint(10), - NullPropagation(conf), + NullPropagation, ConstantFolding, BooleanSimplification, - OptimizeIn(conf)) :: Nil + OptimizeIn) :: Nil } val testRelation = LocalRelation('a.int, 'b.int, 'c.int) @@ -159,16 +159,20 @@ class OptimizeInSuite extends PlanTest { .where(In(UnresolvedAttribute("a"), Seq(Literal(1), Literal(2), Literal(3)))) .analyze - val notOptimizedPlan = OptimizeIn(conf)(plan) - comparePlans(notOptimizedPlan, plan) + withSQLConf(OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> "10") { + val notOptimizedPlan = OptimizeIn(plan) + comparePlans(notOptimizedPlan, plan) + } // Reduce the threshold to turning into InSet. - val optimizedPlan = OptimizeIn(conf.copy(OPTIMIZER_INSET_CONVERSION_THRESHOLD -> 2))(plan) - optimizedPlan match { - case Filter(cond, _) - if cond.isInstanceOf[InSet] && cond.asInstanceOf[InSet].getHSet().size == 3 => - // pass - case _ => fail("Unexpected result for OptimizedIn") + withSQLConf(OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> "2") { + val optimizedPlan = OptimizeIn(plan) + optimizedPlan match { + case Filter(cond, _) + if cond.isInstanceOf[InSet] && cond.asInstanceOf[InSet].getHSet().size == 3 => + // pass + case _ => fail("Unexpected result for OptimizedIn") + } } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala index a23d6266b284..ada6e2a43ea0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala @@ -24,28 +24,46 @@ import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.statsEstimation.{StatsEstimationTestBase, StatsTestPlan} -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf._ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBase { - override val conf = new SQLConf().copy( - CBO_ENABLED -> true, - JOIN_REORDER_ENABLED -> true, - JOIN_REORDER_DP_STAR_FILTER -> true) - object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Operator Optimizations", FixedPoint(100), CombineFilters, PushDownPredicate, - ReorderJoin(conf), + ReorderJoin, PushPredicateThroughJoin, ColumnPruning, CollapseProject) :: - Batch("Join Reorder", Once, - CostBasedJoinReorder(conf)) :: Nil + Batch("Join Reorder", Once, + CostBasedJoinReorder) :: Nil + } + + var originalConfCBOEnabled = false + var originalConfJoinReorderEnabled = false + var originalConfJoinReorderDPStarFilter = false + + override def beforeAll(): Unit = { + super.beforeAll() + originalConfCBOEnabled = conf.cboEnabled + originalConfJoinReorderEnabled = conf.joinReorderEnabled + originalConfJoinReorderDPStarFilter = conf.joinReorderDPStarFilter + conf.setConf(CBO_ENABLED, true) + conf.setConf(JOIN_REORDER_ENABLED, true) + conf.setConf(JOIN_REORDER_DP_STAR_FILTER, true) + } + + override def afterAll(): Unit = { + try { + conf.setConf(CBO_ENABLED, originalConfCBOEnabled) + conf.setConf(JOIN_REORDER_ENABLED, originalConfJoinReorderEnabled) + conf.setConf(JOIN_REORDER_DP_STAR_FILTER, originalConfJoinReorderDPStarFilter) + } finally { + super.afterAll() + } } private val columnInfo: AttributeMap[ColumnStat] = AttributeMap(Seq( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala index 605c01b7220d..777c5637201e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala @@ -24,19 +24,36 @@ import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.statsEstimation.{StatsEstimationTestBase, StatsTestPlan} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.{CASE_SENSITIVE, STARSCHEMA_DETECTION} +import org.apache.spark.sql.internal.SQLConf._ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { - override val conf = new SQLConf().copy(CASE_SENSITIVE -> true, STARSCHEMA_DETECTION -> true) + var originalConfStarSchemaDetection = false + var originalConfCBOEnabled = true + + override def beforeAll(): Unit = { + super.beforeAll() + originalConfStarSchemaDetection = conf.starSchemaDetection + originalConfCBOEnabled = conf.cboEnabled + conf.setConf(STARSCHEMA_DETECTION, true) + conf.setConf(CBO_ENABLED, false) + } + + override def afterAll(): Unit = { + try { + conf.setConf(STARSCHEMA_DETECTION, originalConfStarSchemaDetection) + conf.setConf(CBO_ENABLED, originalConfCBOEnabled) + } finally { + super.afterAll() + } + } object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Operator Optimizations", FixedPoint(100), CombineFilters, PushDownPredicate, - ReorderJoin(conf), + ReorderJoin, PushPredicateThroughJoin, ColumnPruning, CollapseProject) :: Nil diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala index 0a18858350e1..3634accf1ec2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala @@ -37,7 +37,7 @@ class ComplexTypesSuite extends PlanTest{ Batch("collapse projections", FixedPoint(10), CollapseProject) :: Batch("Constant Folding", FixedPoint(10), - NullPropagation(conf), + NullPropagation, ConstantFolding, BooleanSimplification, SimplifyConditionals, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index e9679d336150..5389bf3389da 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -31,8 +31,8 @@ import org.apache.spark.sql.internal.SQLConf */ trait PlanTest extends SparkFunSuite with PredicateHelper { - // TODO(gatorsmile): remove this from PlanTest and all the analyzer/optimizer rules - protected val conf = new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true) + // TODO(gatorsmile): remove this from PlanTest and all the analyzer rules + protected def conf = SQLConf.get /** * Since attribute references are given globally unique ids during analysis, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala index 3c046ce49428..5cfad9126986 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala @@ -38,12 +38,10 @@ import org.apache.spark.sql.internal.SQLConf * 3. aggregate function on partition columns which have same result w or w/o DISTINCT keyword. * e.g. SELECT col1, Max(col2) FROM tbl GROUP BY col1. */ -case class OptimizeMetadataOnlyQuery( - catalog: SessionCatalog, - conf: SQLConf) extends Rule[LogicalPlan] { +case class OptimizeMetadataOnlyQuery(catalog: SessionCatalog) extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { - if (!conf.optimizerMetadataOnly) { + if (!SQLConf.get.optimizerMetadataOnly) { return plan } @@ -106,7 +104,7 @@ case class OptimizeMetadataOnlyQuery( val caseInsensitiveProperties = CaseInsensitiveMap(relation.tableMeta.storage.properties) val timeZoneId = caseInsensitiveProperties.get(DateTimeUtils.TIMEZONE_OPTION) - .getOrElse(conf.sessionLocalTimeZone) + .getOrElse(SQLConf.get.sessionLocalTimeZone) val partitionData = catalog.listPartitions(relation.tableMeta.identifier).map { p => InternalRow.fromSeq(partAttrs.map { attr => Cast(Literal(p.spec(attr.name)), attr.dataType, Option(timeZoneId)).eval() 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 index 1de4f508b89a..00ff4c8ac310 100644 --- 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 @@ -22,16 +22,14 @@ import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions import org.apache.spark.sql.execution.python.ExtractPythonUDFFromAggregate -import org.apache.spark.sql.internal.SQLConf class SparkOptimizer( catalog: SessionCatalog, - conf: SQLConf, experimentalMethods: ExperimentalMethods) - extends Optimizer(catalog, conf) { + extends Optimizer(catalog) { override def batches: Seq[Batch] = (preOptimizationBatches ++ super.batches :+ - Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog, conf)) :+ + Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+ Batch("Extract Python UDF from Aggregate", Once, ExtractPythonUDFFromAggregate) :+ Batch("Prune File Source Table Partitions", Once, PruneFileSourcePartitions)) ++ postHocOptimizationBatches :+ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 9d0148117fad..72d0ddc62303 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -208,7 +208,7 @@ abstract class BaseSessionStateBuilder( * Note: this depends on the `conf`, `catalog` and `experimentalMethods` fields. */ protected def optimizer: Optimizer = { - new SparkOptimizer(catalog, conf, experimentalMethods) { + new SparkOptimizer(catalog, experimentalMethods) { override def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = super.extendedOperatorOptimizationRules ++ customOperatorOptimizationRules } From 14a3bb3a008c302aac908d7deaf0942a98c63be7 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Thu, 6 Jul 2017 14:47:22 +0800 Subject: [PATCH 049/125] [SPARK-21312][SQL] correct offsetInBytes in UnsafeRow.writeToStream ## What changes were proposed in this pull request? Corrects offsetInBytes calculation in UnsafeRow.writeToStream. Known failures include writes to some DataSources that have own SparkPlan implementations and cause EXCHANGE in writes. ## How was this patch tested? Extended UnsafeRowSuite.writeToStream to include an UnsafeRow over byte array having non-zero offset. Author: Sumedh Wale Closes #18535 from sumwale/SPARK-21312. --- .../spark/sql/catalyst/expressions/UnsafeRow.java | 2 +- .../scala/org/apache/spark/sql/UnsafeRowSuite.scala | 13 +++++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) 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 86de90984ca0..56994fafe064 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 @@ -550,7 +550,7 @@ public void copyFrom(UnsafeRow row) { */ public void writeToStream(OutputStream out, byte[] writeBuffer) throws IOException { if (baseObject instanceof byte[]) { - int offsetInByteArray = (int) (Platform.BYTE_ARRAY_OFFSET - baseOffset); + int offsetInByteArray = (int) (baseOffset - Platform.BYTE_ARRAY_OFFSET); out.write((byte[]) baseObject, offsetInByteArray, sizeInBytes); } else { int dataRemaining = sizeInBytes; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala index a32763db054f..a5f904c621e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala @@ -101,9 +101,22 @@ class UnsafeRowSuite extends SparkFunSuite { MemoryAllocator.UNSAFE.free(offheapRowPage) } } + val (bytesFromArrayBackedRowWithOffset, field0StringFromArrayBackedRowWithOffset) = { + val baos = new ByteArrayOutputStream() + val numBytes = arrayBackedUnsafeRow.getSizeInBytes + val bytesWithOffset = new Array[Byte](numBytes + 100) + System.arraycopy(arrayBackedUnsafeRow.getBaseObject.asInstanceOf[Array[Byte]], 0, + bytesWithOffset, 100, numBytes) + val arrayBackedRow = new UnsafeRow(arrayBackedUnsafeRow.numFields()) + arrayBackedRow.pointTo(bytesWithOffset, Platform.BYTE_ARRAY_OFFSET + 100, numBytes) + arrayBackedRow.writeToStream(baos, null) + (baos.toByteArray, arrayBackedRow.getString(0)) + } assert(bytesFromArrayBackedRow === bytesFromOffheapRow) assert(field0StringFromArrayBackedRow === field0StringFromOffheapRow) + assert(bytesFromArrayBackedRow === bytesFromArrayBackedRowWithOffset) + assert(field0StringFromArrayBackedRow === field0StringFromArrayBackedRowWithOffset) } test("calling getDouble() and getFloat() on null columns") { From 60043f22458668ac7ecba94fa78953f23a6bdcec Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 6 Jul 2017 00:20:26 -0700 Subject: [PATCH 050/125] [SS][MINOR] Fix flaky test in DatastreamReaderWriterSuite. temp checkpoint dir should be deleted ## What changes were proposed in this pull request? Stopping query while it is being initialized can throw interrupt exception, in which case temporary checkpoint directories will not be deleted, and the test will fail. Author: Tathagata Das Closes #18442 from tdas/DatastreamReaderWriterSuite-fix. --- .../spark/sql/streaming/test/DataStreamReaderWriterSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala index 3de0ae67a389..e8a6202b8adc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -641,6 +641,7 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { test("temp checkpoint dir should be deleted if a query is stopped without errors") { import testImplicits._ val query = MemoryStream[Int].toDS.writeStream.format("console").start() + query.processAllAvailable() val checkpointDir = new Path( query.asInstanceOf[StreamingQueryWrapper].streamingQuery.resolvedCheckpointRoot) val fs = checkpointDir.getFileSystem(spark.sessionState.newHadoopConf()) From 5800144a54f5c0180ccf67392f32c3e8a51119b1 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 6 Jul 2017 15:32:49 +0800 Subject: [PATCH 051/125] [SPARK-21012][SUBMIT] Add glob support for resources adding to Spark Current "--jars (spark.jars)", "--files (spark.files)", "--py-files (spark.submit.pyFiles)" and "--archives (spark.yarn.dist.archives)" only support non-glob path. This is OK for most of the cases, but when user requires to add more jars, files into Spark, it is too verbose to list one by one. So here propose to add glob path support for resources. Also improving the code of downloading resources. ## How was this patch tested? UT added, also verified manually in local cluster. Author: jerryshao Closes #18235 from jerryshao/SPARK-21012. --- .../org/apache/spark/deploy/SparkSubmit.scala | 166 ++++++++++++++---- .../spark/deploy/SparkSubmitArguments.scala | 2 +- .../spark/deploy/SparkSubmitSuite.scala | 68 ++++++- docs/configuration.md | 6 +- 4 files changed, 196 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index d13fb4193970..abde04062c4b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -17,17 +17,21 @@ package org.apache.spark.deploy -import java.io.{File, IOException} +import java.io._ import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowableException} import java.net.URL import java.nio.file.Files -import java.security.PrivilegedExceptionAction +import java.security.{KeyStore, PrivilegedExceptionAction} +import java.security.cert.X509Certificate import java.text.ParseException +import javax.net.ssl._ import scala.annotation.tailrec import scala.collection.mutable.{ArrayBuffer, HashMap, Map} import scala.util.Properties +import com.google.common.io.ByteStreams +import org.apache.commons.io.FileUtils import org.apache.commons.lang3.StringUtils import org.apache.hadoop.conf.{Configuration => HadoopConfiguration} import org.apache.hadoop.fs.{FileSystem, Path} @@ -310,33 +314,33 @@ object SparkSubmit extends CommandLineUtils { RPackageUtils.checkAndBuildRPackage(args.jars, printStream, args.verbose) } - // In client mode, download remote files. - if (deployMode == CLIENT) { - val hadoopConf = new HadoopConfiguration() - args.primaryResource = Option(args.primaryResource).map(downloadFile(_, hadoopConf)).orNull - args.jars = Option(args.jars).map(downloadFileList(_, hadoopConf)).orNull - args.pyFiles = Option(args.pyFiles).map(downloadFileList(_, hadoopConf)).orNull - args.files = Option(args.files).map(downloadFileList(_, hadoopConf)).orNull - } - - // Require all python files to be local, so we can add them to the PYTHONPATH - // In YARN cluster mode, python files are distributed as regular files, which can be non-local. - // In Mesos cluster mode, non-local python files are automatically downloaded by Mesos. - if (args.isPython && !isYarnCluster && !isMesosCluster) { - if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) { - printErrorAndExit(s"Only local python files are supported: ${args.primaryResource}") + val hadoopConf = new HadoopConfiguration() + val targetDir = Files.createTempDirectory("tmp").toFile + // scalastyle:off runtimeaddshutdownhook + Runtime.getRuntime.addShutdownHook(new Thread() { + override def run(): Unit = { + FileUtils.deleteQuietly(targetDir) } - val nonLocalPyFiles = Utils.nonLocalPaths(args.pyFiles).mkString(",") - if (nonLocalPyFiles.nonEmpty) { - printErrorAndExit(s"Only local additional python files are supported: $nonLocalPyFiles") - } - } + }) + // scalastyle:on runtimeaddshutdownhook - // Require all R files to be local - if (args.isR && !isYarnCluster && !isMesosCluster) { - if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) { - printErrorAndExit(s"Only local R files are supported: ${args.primaryResource}") - } + // Resolve glob path for different resources. + args.jars = Option(args.jars).map(resolveGlobPaths(_, hadoopConf)).orNull + args.files = Option(args.files).map(resolveGlobPaths(_, hadoopConf)).orNull + args.pyFiles = Option(args.pyFiles).map(resolveGlobPaths(_, hadoopConf)).orNull + args.archives = Option(args.archives).map(resolveGlobPaths(_, hadoopConf)).orNull + + // In client mode, download remote files. + if (deployMode == CLIENT) { + args.primaryResource = Option(args.primaryResource).map { + downloadFile(_, targetDir, args.sparkProperties, hadoopConf) + }.orNull + args.jars = Option(args.jars).map { + downloadFileList(_, targetDir, args.sparkProperties, hadoopConf) + }.orNull + args.pyFiles = Option(args.pyFiles).map { + downloadFileList(_, targetDir, args.sparkProperties, hadoopConf) + }.orNull } // The following modes are not supported or applicable @@ -841,36 +845,132 @@ object SparkSubmit extends CommandLineUtils { * Download a list of remote files to temp local files. If the file is local, the original file * will be returned. * @param fileList A comma separated file list. + * @param targetDir A temporary directory for which downloaded files + * @param sparkProperties Spark properties * @return A comma separated local files list. */ private[deploy] def downloadFileList( fileList: String, + targetDir: File, + sparkProperties: Map[String, String], hadoopConf: HadoopConfiguration): String = { require(fileList != null, "fileList cannot be null.") - fileList.split(",").map(downloadFile(_, hadoopConf)).mkString(",") + fileList.split(",") + .map(downloadFile(_, targetDir, sparkProperties, hadoopConf)) + .mkString(",") } /** * Download a file from the remote to a local temporary directory. If the input path points to * a local path, returns it with no operation. + * @param path A file path from where the files will be downloaded. + * @param targetDir A temporary directory for which downloaded files + * @param sparkProperties Spark properties + * @return A comma separated local files list. */ - private[deploy] def downloadFile(path: String, hadoopConf: HadoopConfiguration): String = { + private[deploy] def downloadFile( + path: String, + targetDir: File, + sparkProperties: Map[String, String], + hadoopConf: HadoopConfiguration): String = { require(path != null, "path cannot be null.") val uri = Utils.resolveURI(path) uri.getScheme match { - case "file" | "local" => - path + case "file" | "local" => path + case "http" | "https" | "ftp" => + val uc = uri.toURL.openConnection() + uc match { + case https: HttpsURLConnection => + val trustStore = sparkProperties.get("spark.ssl.fs.trustStore") + .orElse(sparkProperties.get("spark.ssl.trustStore")) + val trustStorePwd = sparkProperties.get("spark.ssl.fs.trustStorePassword") + .orElse(sparkProperties.get("spark.ssl.trustStorePassword")) + .map(_.toCharArray) + .orNull + val protocol = sparkProperties.get("spark.ssl.fs.protocol") + .orElse(sparkProperties.get("spark.ssl.protocol")) + if (protocol.isEmpty) { + printErrorAndExit("spark ssl protocol is required when enabling SSL connection.") + } + + val trustStoreManagers = trustStore.map { t => + var input: InputStream = null + try { + input = new FileInputStream(new File(t)) + val ks = KeyStore.getInstance(KeyStore.getDefaultType) + ks.load(input, trustStorePwd) + val tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm) + tmf.init(ks) + tmf.getTrustManagers + } finally { + if (input != null) { + input.close() + input = null + } + } + }.getOrElse { + Array({ + new X509TrustManager { + override def getAcceptedIssuers: Array[X509Certificate] = null + override def checkClientTrusted( + x509Certificates: Array[X509Certificate], s: String) {} + override def checkServerTrusted( + x509Certificates: Array[X509Certificate], s: String) {} + }: TrustManager + }) + } + val sslContext = SSLContext.getInstance(protocol.get) + sslContext.init(null, trustStoreManagers, null) + https.setSSLSocketFactory(sslContext.getSocketFactory) + https.setHostnameVerifier(new HostnameVerifier { + override def verify(s: String, sslSession: SSLSession): Boolean = false + }) + + case _ => + } + uc.setConnectTimeout(60 * 1000) + uc.setReadTimeout(60 * 1000) + uc.connect() + val in = uc.getInputStream + val fileName = new Path(uri).getName + val tempFile = new File(targetDir, fileName) + val out = new FileOutputStream(tempFile) + // scalastyle:off println + printStream.println(s"Downloading ${uri.toString} to ${tempFile.getAbsolutePath}.") + // scalastyle:on println + try { + ByteStreams.copy(in, out) + } finally { + in.close() + out.close() + } + tempFile.toURI.toString case _ => val fs = FileSystem.get(uri, hadoopConf) - val tmpFile = new File(Files.createTempDirectory("tmp").toFile, uri.getPath) + val tmpFile = new File(targetDir, new Path(uri).getName) // scalastyle:off println printStream.println(s"Downloading ${uri.toString} to ${tmpFile.getAbsolutePath}.") // scalastyle:on println fs.copyToLocalFile(new Path(uri), new Path(tmpFile.getAbsolutePath)) - Utils.resolveURI(tmpFile.getAbsolutePath).toString + tmpFile.toURI.toString } } + + private def resolveGlobPaths(paths: String, hadoopConf: HadoopConfiguration): String = { + require(paths != null, "paths cannot be null.") + paths.split(",").map(_.trim).filter(_.nonEmpty).flatMap { path => + val uri = Utils.resolveURI(path) + uri.getScheme match { + case "local" | "http" | "https" | "ftp" => Array(path) + case _ => + val fs = FileSystem.get(uri, hadoopConf) + Option(fs.globStatus(new Path(uri))).map { status => + status.filter(_.isFile).map(_.getPath.toUri.toString) + }.getOrElse(Array(path)) + } + }.mkString(",") + } } /** Provides utility functions to be used inside SparkSubmit. */ diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 7800d3d624e3..fd1521193fde 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -520,7 +520,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | (Default: client). | --class CLASS_NAME Your application's main class (for Java / Scala apps). | --name NAME A name of your application. - | --jars JARS Comma-separated list of local jars to include on the driver + | --jars JARS Comma-separated list of jars to include on the driver | and executor classpaths. | --packages Comma-separated list of maven coordinates of jars to include | on the driver and executor classpaths. Will search the local diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index b089357e7b86..97357cdbb608 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -20,12 +20,14 @@ package org.apache.spark.deploy import java.io._ import java.net.URI import java.nio.charset.StandardCharsets +import java.nio.file.Files +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.io.Source import com.google.common.io.ByteStreams -import org.apache.commons.io.{FilenameUtils, FileUtils} +import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.scalatest.{BeforeAndAfterEach, Matchers} @@ -42,7 +44,6 @@ import org.apache.spark.TestUtils.JavaSourceFromString import org.apache.spark.scheduler.EventLoggingListener import org.apache.spark.util.{CommandLineUtils, ResetSystemProperties, Utils} - trait TestPrematureExit { suite: SparkFunSuite => @@ -726,6 +727,47 @@ class SparkSubmitSuite Utils.unionFileLists(None, Option("/tmp/a.jar")) should be (Set("/tmp/a.jar")) Utils.unionFileLists(Option("/tmp/a.jar"), None) should be (Set("/tmp/a.jar")) } + + test("support glob path") { + val tmpJarDir = Utils.createTempDir() + val jar1 = TestUtils.createJarWithFiles(Map("test.resource" -> "1"), tmpJarDir) + val jar2 = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"), tmpJarDir) + + val tmpFileDir = Utils.createTempDir() + val file1 = File.createTempFile("tmpFile1", "", tmpFileDir) + val file2 = File.createTempFile("tmpFile2", "", tmpFileDir) + + val tmpPyFileDir = Utils.createTempDir() + val pyFile1 = File.createTempFile("tmpPy1", ".py", tmpPyFileDir) + val pyFile2 = File.createTempFile("tmpPy2", ".egg", tmpPyFileDir) + + val tmpArchiveDir = Utils.createTempDir() + val archive1 = File.createTempFile("archive1", ".zip", tmpArchiveDir) + val archive2 = File.createTempFile("archive2", ".zip", tmpArchiveDir) + + val args = Seq( + "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), + "--name", "testApp", + "--master", "yarn", + "--deploy-mode", "client", + "--jars", s"${tmpJarDir.getAbsolutePath}/*.jar", + "--files", s"${tmpFileDir.getAbsolutePath}/tmpFile*", + "--py-files", s"${tmpPyFileDir.getAbsolutePath}/tmpPy*", + "--archives", s"${tmpArchiveDir.getAbsolutePath}/*.zip", + jar2.toString) + + val appArgs = new SparkSubmitArguments(args) + val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs)._3 + sysProps("spark.yarn.dist.jars").split(",").toSet should be + (Set(jar1.toURI.toString, jar2.toURI.toString)) + sysProps("spark.yarn.dist.files").split(",").toSet should be + (Set(file1.toURI.toString, file2.toURI.toString)) + sysProps("spark.submit.pyFiles").split(",").toSet should be + (Set(pyFile1.getAbsolutePath, pyFile2.getAbsolutePath)) + sysProps("spark.yarn.dist.archives").split(",").toSet should be + (Set(archive1.toURI.toString, archive2.toURI.toString)) + } + // scalastyle:on println private def checkDownloadedFile(sourcePath: String, outputPath: String): Unit = { @@ -738,7 +780,7 @@ class SparkSubmitSuite assert(outputUri.getScheme === "file") // The path and filename are preserved. - assert(outputUri.getPath.endsWith(sourceUri.getPath)) + assert(outputUri.getPath.endsWith(new Path(sourceUri).getName)) assert(FileUtils.readFileToString(new File(outputUri.getPath)) === FileUtils.readFileToString(new File(sourceUri.getPath))) } @@ -752,25 +794,29 @@ class SparkSubmitSuite test("downloadFile - invalid url") { intercept[IOException] { - SparkSubmit.downloadFile("abc:/my/file", new Configuration()) + SparkSubmit.downloadFile( + "abc:/my/file", Utils.createTempDir(), mutable.Map.empty, new Configuration()) } } test("downloadFile - file doesn't exist") { val hadoopConf = new Configuration() + val tmpDir = Utils.createTempDir() // Set s3a implementation to local file system for testing. hadoopConf.set("fs.s3a.impl", "org.apache.spark.deploy.TestFileSystem") // Disable file system impl cache to make sure the test file system is picked up. hadoopConf.set("fs.s3a.impl.disable.cache", "true") intercept[FileNotFoundException] { - SparkSubmit.downloadFile("s3a:/no/such/file", hadoopConf) + SparkSubmit.downloadFile("s3a:/no/such/file", tmpDir, mutable.Map.empty, hadoopConf) } } test("downloadFile does not download local file") { // empty path is considered as local file. - assert(SparkSubmit.downloadFile("", new Configuration()) === "") - assert(SparkSubmit.downloadFile("/local/file", new Configuration()) === "/local/file") + val tmpDir = Files.createTempDirectory("tmp").toFile + assert(SparkSubmit.downloadFile("", tmpDir, mutable.Map.empty, new Configuration()) === "") + assert(SparkSubmit.downloadFile("/local/file", tmpDir, mutable.Map.empty, + new Configuration()) === "/local/file") } test("download one file to local") { @@ -779,12 +825,14 @@ class SparkSubmitSuite val content = "hello, world" FileUtils.write(jarFile, content) val hadoopConf = new Configuration() + val tmpDir = Files.createTempDirectory("tmp").toFile // Set s3a implementation to local file system for testing. hadoopConf.set("fs.s3a.impl", "org.apache.spark.deploy.TestFileSystem") // Disable file system impl cache to make sure the test file system is picked up. hadoopConf.set("fs.s3a.impl.disable.cache", "true") val sourcePath = s"s3a://${jarFile.getAbsolutePath}" - val outputPath = SparkSubmit.downloadFile(sourcePath, hadoopConf) + val outputPath = + SparkSubmit.downloadFile(sourcePath, tmpDir, mutable.Map.empty, hadoopConf) checkDownloadedFile(sourcePath, outputPath) deleteTempOutputFile(outputPath) } @@ -795,12 +843,14 @@ class SparkSubmitSuite val content = "hello, world" FileUtils.write(jarFile, content) val hadoopConf = new Configuration() + val tmpDir = Files.createTempDirectory("tmp").toFile // Set s3a implementation to local file system for testing. hadoopConf.set("fs.s3a.impl", "org.apache.spark.deploy.TestFileSystem") // Disable file system impl cache to make sure the test file system is picked up. hadoopConf.set("fs.s3a.impl.disable.cache", "true") val sourcePaths = Seq("/local/file", s"s3a://${jarFile.getAbsolutePath}") - val outputPaths = SparkSubmit.downloadFileList(sourcePaths.mkString(","), hadoopConf).split(",") + val outputPaths = SparkSubmit.downloadFileList( + sourcePaths.mkString(","), tmpDir, mutable.Map.empty, hadoopConf).split(",") assert(outputPaths.length === sourcePaths.length) sourcePaths.zip(outputPaths).foreach { case (sourcePath, outputPath) => diff --git a/docs/configuration.md b/docs/configuration.md index c785a664c67b..7dc23e441a7b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -422,21 +422,21 @@ Apart from these, the following properties are also available, and may be useful spark.files - Comma-separated list of files to be placed in the working directory of each executor. + Comma-separated list of files to be placed in the working directory of each executor. Globs are allowed. spark.submit.pyFiles - Comma-separated list of .zip, .egg, or .py files to place on the PYTHONPATH for Python apps. + Comma-separated list of .zip, .egg, or .py files to place on the PYTHONPATH for Python apps. Globs are allowed. spark.jars - Comma-separated list of local jars to include on the driver and executor classpaths. + Comma-separated list of jars to include on the driver and executor classpaths. Globs are allowed. From 6ff05a66fe83e721063efe5c28d2ffeb850fecc7 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 6 Jul 2017 15:47:09 +0800 Subject: [PATCH 052/125] [SPARK-20703][SQL] Associate metrics with data writes onto DataFrameWriter operations ## What changes were proposed in this pull request? Right now in the UI, after SPARK-20213, we can show the operations to write data out. However, there is no way to associate metrics with data writes. We should show relative metrics on the operations. #### Supported commands This change supports updating metrics for file-based data writing operations, including `InsertIntoHadoopFsRelationCommand`, `InsertIntoHiveTable`. Supported metrics: * number of written files * number of dynamic partitions * total bytes of written data * total number of output rows * average writing data out time (ms) * (TODO) min/med/max number of output rows per file/partition * (TODO) min/med/max bytes of written data per file/partition #### Commands not supported `InsertIntoDataSourceCommand`, `SaveIntoDataSourceCommand`: The two commands uses DataSource APIs to write data out, i.e., the logic of writing data out is delegated to the DataSource implementations, such as `InsertableRelation.insert` and `CreatableRelationProvider.createRelation`. So we can't obtain metrics from delegated methods for now. `CreateHiveTableAsSelectCommand`, `CreateDataSourceTableAsSelectCommand` : The two commands invokes other commands to write data out. The invoked commands can even write to non file-based data source. We leave them as future TODO. #### How to update metrics of writing files out A `RunnableCommand` which wants to update metrics, needs to override its `metrics` and provide the metrics data structure to `ExecutedCommandExec`. The metrics are prepared during the execution of `FileFormatWriter`. The callback function passed to `FileFormatWriter` will accept the metrics and update accordingly. There is a metrics updating function in `RunnableCommand`. In runtime, the function will be bound to the spark context and `metrics` of `ExecutedCommandExec` and pass to `FileFormatWriter`. ## How was this patch tested? Updated unit tests. Author: Liang-Chi Hsieh Closes #18159 from viirya/SPARK-20703-2. --- .../scala/org/apache/spark/util/Utils.scala | 9 ++ .../command/DataWritingCommand.scala | 75 ++++++++++ .../sql/execution/command/commands.scala | 12 ++ .../datasources/FileFormatWriter.scala | 121 ++++++++++++--- .../InsertIntoHadoopFsRelationCommand.scala | 18 ++- .../sql/sources/PartitionedWriteSuite.scala | 21 +-- .../hive/execution/InsertIntoHiveTable.scala | 8 +- .../sql/hive/execution/SQLMetricsSuite.scala | 139 ++++++++++++++++++ 8 files changed, 362 insertions(+), 41 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 26f61e25da4d..b4caf68f0afa 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1002,6 +1002,15 @@ private[spark] object Utils extends Logging { } } + /** + * Lists files recursively. + */ + def recursiveList(f: File): Array[File] = { + require(f.isDirectory) + val current = f.listFiles + current ++ current.filter(_.isDirectory).flatMap(recursiveList) + } + /** * Delete a file or directory and its contents recursively. * Don't follow directories if they are symlinks. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala new file mode 100644 index 000000000000..0c381a2c0298 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.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.execution.command + +import org.apache.spark.SparkContext +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** + * A special `RunnableCommand` which writes data out and updates metrics. + */ +trait DataWritingCommand extends RunnableCommand { + + override lazy val metrics: Map[String, SQLMetric] = { + val sparkContext = SparkContext.getActive.get + Map( + "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), + "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), + "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part") + ) + } + + /** + * Callback function that update metrics collected from the writing operation. + */ + protected def updateWritingMetrics(writeSummaries: Seq[ExecutedWriteSummary]): Unit = { + val sparkContext = SparkContext.getActive.get + var numPartitions = 0 + var numFiles = 0 + var totalNumBytes: Long = 0L + var totalNumOutput: Long = 0L + var totalWritingTime: Long = 0L + + writeSummaries.foreach { summary => + numPartitions += summary.updatedPartitions.size + numFiles += summary.numOutputFile + totalNumBytes += summary.numOutputBytes + totalNumOutput += summary.numOutputRows + totalWritingTime += summary.totalWritingTime + } + + val avgWritingTime = if (numFiles > 0) { + (totalWritingTime / numFiles).toLong + } else { + 0L + } + + metrics("avgTime").add(avgWritingTime) + metrics("numFiles").add(numFiles) + metrics("numOutputBytes").add(totalNumBytes) + metrics("numOutputRows").add(totalNumOutput) + metrics("numParts").add(numPartitions) + + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toList) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 81bc93e7ebcf..7cd4baef89e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.debug._ +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types._ @@ -37,6 +38,11 @@ import org.apache.spark.sql.types._ * wrapped in `ExecutedCommand` during execution. */ trait RunnableCommand extends logical.Command { + + // The map used to record the metrics of running the command. This will be passed to + // `ExecutedCommand` during query planning. + lazy val metrics: Map[String, SQLMetric] = Map.empty + def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { throw new NotImplementedError } @@ -49,8 +55,14 @@ trait RunnableCommand extends logical.Command { /** * A physical operator that executes the run method of a `RunnableCommand` and * saves the result to prevent multiple executions. + * + * @param cmd the `RunnableCommand` this operator will run. + * @param children the children physical plans ran by the `RunnableCommand`. */ case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) extends SparkPlan { + + override lazy val metrics: Map[String, SQLMetric] = cmd.metrics + /** * A concrete command should override this lazy field to wrap up any side effects caused by the * command or any other computation that should be evaluated exactly once. The value of this field diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index 0daffa93b474..64866630623a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -22,7 +22,7 @@ import java.util.{Date, UUID} import scala.collection.mutable import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl @@ -82,7 +82,7 @@ object FileFormatWriter extends Logging { } /** The result of a successful write task. */ - private case class WriteTaskResult(commitMsg: TaskCommitMessage, updatedPartitions: Set[String]) + private case class WriteTaskResult(commitMsg: TaskCommitMessage, summary: ExecutedWriteSummary) /** * Basic work flow of this command is: @@ -104,7 +104,7 @@ object FileFormatWriter extends Logging { hadoopConf: Configuration, partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], - refreshFunction: (Seq[TablePartitionSpec]) => Unit, + refreshFunction: (Seq[ExecutedWriteSummary]) => Unit, options: Map[String, String]): Unit = { val job = Job.getInstance(hadoopConf) @@ -196,12 +196,10 @@ object FileFormatWriter extends Logging { }) val commitMsgs = ret.map(_.commitMsg) - val updatedPartitions = ret.flatMap(_.updatedPartitions) - .distinct.map(PartitioningUtils.parsePathFragment) committer.commitJob(job, commitMsgs) logInfo(s"Job ${job.getJobID} committed.") - refreshFunction(updatedPartitions) + refreshFunction(ret.map(_.summary)) } catch { case cause: Throwable => logError(s"Aborting job ${job.getJobID}.", cause) committer.abortJob(job) @@ -247,9 +245,9 @@ object FileFormatWriter extends Logging { try { Utils.tryWithSafeFinallyAndFailureCallbacks(block = { // Execute the task to write rows out and commit the task. - val outputPartitions = writeTask.execute(iterator) + val summary = writeTask.execute(iterator) writeTask.releaseResources() - WriteTaskResult(committer.commitTask(taskAttemptContext), outputPartitions) + WriteTaskResult(committer.commitTask(taskAttemptContext), summary) })(catchBlock = { // If there is an error, release resource and then abort the task try { @@ -273,12 +271,36 @@ object FileFormatWriter extends Logging { * automatically trigger task aborts. */ private trait ExecuteWriteTask { + /** - * Writes data out to files, and then returns the list of partition strings written out. - * The list of partitions is sent back to the driver and used to update the catalog. + * The data structures used to measure metrics during writing. */ - def execute(iterator: Iterator[InternalRow]): Set[String] + protected var totalWritingTime: Long = 0L + protected var timeOnCurrentFile: Long = 0L + protected var numOutputRows: Long = 0L + protected var numOutputBytes: Long = 0L + + /** + * Writes data out to files, and then returns the summary of relative information which + * includes the list of partition strings written out. The list of partitions is sent back + * to the driver and used to update the catalog. Other information will be sent back to the + * driver too and used to update the metrics in UI. + */ + def execute(iterator: Iterator[InternalRow]): ExecutedWriteSummary def releaseResources(): Unit + + /** + * A helper function used to determine the size in bytes of a written file. + */ + protected def getFileSize(conf: Configuration, filePath: String): Long = { + if (filePath != null) { + val path = new Path(filePath) + val fs = path.getFileSystem(conf) + fs.getFileStatus(path).getLen() + } else { + 0L + } + } } /** Writes data to a single directory (used for non-dynamic-partition writes). */ @@ -288,24 +310,26 @@ object FileFormatWriter extends Logging { committer: FileCommitProtocol) extends ExecuteWriteTask { private[this] var currentWriter: OutputWriter = _ + private[this] var currentPath: String = _ private def newOutputWriter(fileCounter: Int): Unit = { val ext = description.outputWriterFactory.getFileExtension(taskAttemptContext) - val tmpFilePath = committer.newTaskTempFile( + currentPath = committer.newTaskTempFile( taskAttemptContext, None, f"-c$fileCounter%03d" + ext) currentWriter = description.outputWriterFactory.newInstance( - path = tmpFilePath, + path = currentPath, dataSchema = description.dataColumns.toStructType, context = taskAttemptContext) } - override def execute(iter: Iterator[InternalRow]): Set[String] = { + override def execute(iter: Iterator[InternalRow]): ExecutedWriteSummary = { var fileCounter = 0 var recordsInFile: Long = 0L newOutputWriter(fileCounter) + while (iter.hasNext) { if (description.maxRecordsPerFile > 0 && recordsInFile >= description.maxRecordsPerFile) { fileCounter += 1 @@ -314,21 +338,35 @@ object FileFormatWriter extends Logging { recordsInFile = 0 releaseResources() + numOutputRows += recordsInFile newOutputWriter(fileCounter) } val internalRow = iter.next() + val startTime = System.nanoTime() currentWriter.write(internalRow) + timeOnCurrentFile += (System.nanoTime() - startTime) recordsInFile += 1 } releaseResources() - Set.empty + numOutputRows += recordsInFile + + ExecutedWriteSummary( + updatedPartitions = Set.empty, + numOutputFile = fileCounter + 1, + numOutputBytes = numOutputBytes, + numOutputRows = numOutputRows, + totalWritingTime = totalWritingTime) } override def releaseResources(): Unit = { if (currentWriter != null) { try { + val startTime = System.nanoTime() currentWriter.close() + totalWritingTime += (timeOnCurrentFile + System.nanoTime() - startTime) / 1000 / 1000 + timeOnCurrentFile = 0 + numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) } finally { currentWriter = null } @@ -348,6 +386,8 @@ object FileFormatWriter extends Logging { // currentWriter is initialized whenever we see a new key private var currentWriter: OutputWriter = _ + private var currentPath: String = _ + /** Expressions that given partition columns build a path string like: col1=val/col2=val/... */ private def partitionPathExpression: Seq[Expression] = { desc.partitionColumns.zipWithIndex.flatMap { case (c, i) => @@ -403,19 +443,19 @@ object FileFormatWriter extends Logging { case _ => None } - val path = if (customPath.isDefined) { + currentPath = if (customPath.isDefined) { committer.newTaskTempFileAbsPath(taskAttemptContext, customPath.get, ext) } else { committer.newTaskTempFile(taskAttemptContext, partDir, ext) } currentWriter = desc.outputWriterFactory.newInstance( - path = path, + path = currentPath, dataSchema = desc.dataColumns.toStructType, context = taskAttemptContext) } - override def execute(iter: Iterator[InternalRow]): Set[String] = { + override def execute(iter: Iterator[InternalRow]): ExecutedWriteSummary = { val getPartitionColsAndBucketId = UnsafeProjection.create( desc.partitionColumns ++ desc.bucketIdExpression, desc.allColumns) @@ -429,15 +469,22 @@ object FileFormatWriter extends Logging { // If anything below fails, we should abort the task. var recordsInFile: Long = 0L var fileCounter = 0 + var totalFileCounter = 0 var currentPartColsAndBucketId: UnsafeRow = null val updatedPartitions = mutable.Set[String]() + for (row <- iter) { val nextPartColsAndBucketId = getPartitionColsAndBucketId(row) if (currentPartColsAndBucketId != nextPartColsAndBucketId) { + if (currentPartColsAndBucketId != null) { + totalFileCounter += (fileCounter + 1) + } + // See a new partition or bucket - write to a new partition dir (or a new bucket file). currentPartColsAndBucketId = nextPartColsAndBucketId.copy() logDebug(s"Writing partition: $currentPartColsAndBucketId") + numOutputRows += recordsInFile recordsInFile = 0 fileCounter = 0 @@ -447,6 +494,8 @@ object FileFormatWriter extends Logging { recordsInFile >= desc.maxRecordsPerFile) { // Exceeded the threshold in terms of the number of records per file. // Create a new file by increasing the file counter. + + numOutputRows += recordsInFile recordsInFile = 0 fileCounter += 1 assert(fileCounter < MAX_FILE_COUNTER, @@ -455,18 +504,33 @@ object FileFormatWriter extends Logging { releaseResources() newOutputWriter(currentPartColsAndBucketId, getPartPath, fileCounter, updatedPartitions) } - + val startTime = System.nanoTime() currentWriter.write(getOutputRow(row)) + timeOnCurrentFile += (System.nanoTime() - startTime) recordsInFile += 1 } + if (currentPartColsAndBucketId != null) { + totalFileCounter += (fileCounter + 1) + } releaseResources() - updatedPartitions.toSet + numOutputRows += recordsInFile + + ExecutedWriteSummary( + updatedPartitions = updatedPartitions.toSet, + numOutputFile = totalFileCounter, + numOutputBytes = numOutputBytes, + numOutputRows = numOutputRows, + totalWritingTime = totalWritingTime) } override def releaseResources(): Unit = { if (currentWriter != null) { try { + val startTime = System.nanoTime() currentWriter.close() + totalWritingTime += (timeOnCurrentFile + System.nanoTime() - startTime) / 1000 / 1000 + timeOnCurrentFile = 0 + numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) } finally { currentWriter = null } @@ -474,3 +538,20 @@ object FileFormatWriter extends Logging { } } } + +/** + * Wrapper class for the metrics of writing data out. + * + * @param updatedPartitions the partitions updated during writing data out. Only valid + * for dynamic partition. + * @param numOutputFile the total number of files. + * @param numOutputRows the number of output rows. + * @param numOutputBytes the bytes of output data. + * @param totalWritingTime the total writing time in ms. + */ +case class ExecutedWriteSummary( + updatedPartitions: Set[String], + numOutputFile: Int, + numOutputRows: Long, + numOutputBytes: Long, + totalWritingTime: Long) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index ab26f2affbce..0031567d3d28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -21,6 +21,7 @@ import java.io.IOException import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.spark.SparkContext import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTablePartition} @@ -29,6 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} /** * A command for writing data to a [[HadoopFsRelation]]. Supports both overwriting and appending. @@ -53,7 +55,7 @@ case class InsertIntoHadoopFsRelationCommand( mode: SaveMode, catalogTable: Option[CatalogTable], fileIndex: Option[FileIndex]) - extends RunnableCommand { + extends DataWritingCommand { import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName override def children: Seq[LogicalPlan] = query :: Nil @@ -123,8 +125,16 @@ case class InsertIntoHadoopFsRelationCommand( if (doInsertion) { - // Callback for updating metastore partition metadata after the insertion job completes. - def refreshPartitionsCallback(updatedPartitions: Seq[TablePartitionSpec]): Unit = { + // Callback for updating metric and metastore partition metadata + // after the insertion job completes. + def refreshCallback(summary: Seq[ExecutedWriteSummary]): Unit = { + val updatedPartitions = summary.flatMap(_.updatedPartitions) + .distinct.map(PartitioningUtils.parsePathFragment) + + // Updating metrics. + updateWritingMetrics(summary) + + // Updating metastore partition metadata. if (partitionsTrackedByCatalog) { val newPartitions = updatedPartitions.toSet -- initialMatchingPartitions if (newPartitions.nonEmpty) { @@ -154,7 +164,7 @@ case class InsertIntoHadoopFsRelationCommand( hadoopConf = hadoopConf, partitionColumns = partitionColumns, bucketSpec = bucketSpec, - refreshFunction = refreshPartitionsCallback, + refreshFunction = refreshCallback, options = options) // refresh cached files in FileIndex diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala index a2f3afe3ce23..6f998aa60faf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala @@ -91,15 +91,15 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext { withTempDir { f => spark.range(start = 0, end = 4, step = 1, numPartitions = 1) .write.option("maxRecordsPerFile", 1).mode("overwrite").parquet(f.getAbsolutePath) - assert(recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) + assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) spark.range(start = 0, end = 4, step = 1, numPartitions = 1) .write.option("maxRecordsPerFile", 2).mode("overwrite").parquet(f.getAbsolutePath) - assert(recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 2) + assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 2) spark.range(start = 0, end = 4, step = 1, numPartitions = 1) .write.option("maxRecordsPerFile", -1).mode("overwrite").parquet(f.getAbsolutePath) - assert(recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 1) + assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 1) } } @@ -111,7 +111,7 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext { .option("maxRecordsPerFile", 1) .mode("overwrite") .parquet(f.getAbsolutePath) - assert(recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) + assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) } } @@ -138,14 +138,14 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext { val df = Seq((1, ts)).toDF("i", "ts") withTempPath { f => df.write.partitionBy("ts").parquet(f.getAbsolutePath) - val files = recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) + val files = Utils.recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) assert(files.length == 1) checkPartitionValues(files.head, "2016-12-01 00:00:00") } withTempPath { f => df.write.option(DateTimeUtils.TIMEZONE_OPTION, "GMT") .partitionBy("ts").parquet(f.getAbsolutePath) - val files = recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) + val files = Utils.recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) assert(files.length == 1) // use timeZone option "GMT" to format partition value. checkPartitionValues(files.head, "2016-12-01 08:00:00") @@ -153,18 +153,11 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext { withTempPath { f => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") { df.write.partitionBy("ts").parquet(f.getAbsolutePath) - val files = recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) + val files = Utils.recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) assert(files.length == 1) // if there isn't timeZone option, then use session local timezone. checkPartitionValues(files.head, "2016-12-01 08:00:00") } } } - - /** Lists files recursively. */ - private def recursiveList(f: File): Array[File] = { - require(f.isDirectory) - val current = f.listFiles - current ++ current.filter(_.isDirectory).flatMap(recursiveList) - } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 223d37523239..cd263e8b6df8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -31,14 +31,16 @@ import org.apache.hadoop.hive.ql.exec.TaskRunner import org.apache.hadoop.hive.ql.ErrorMsg import org.apache.hadoop.hive.ql.plan.TableDesc +import org.apache.spark.SparkContext import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.{CommandUtils, RunnableCommand} +import org.apache.spark.sql.execution.command.{CommandUtils, DataWritingCommand} import org.apache.spark.sql.execution.datasources.FileFormatWriter +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} import org.apache.spark.sql.hive.client.{HiveClientImpl, HiveVersion} @@ -80,7 +82,7 @@ case class InsertIntoHiveTable( partition: Map[String, Option[String]], query: LogicalPlan, overwrite: Boolean, - ifPartitionNotExists: Boolean) extends RunnableCommand { + ifPartitionNotExists: Boolean) extends DataWritingCommand { override def children: Seq[LogicalPlan] = query :: Nil @@ -354,7 +356,7 @@ case class InsertIntoHiveTable( hadoopConf = hadoopConf, partitionColumns = partitionAttributes, bucketSpec = None, - refreshFunction = _ => (), + refreshFunction = updateWritingMetrics, options = Map.empty) if (partition.nonEmpty) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala new file mode 100644 index 000000000000..1ef1988d4c60 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import java.io.File + +import org.apache.spark.sql.catalyst.TableIdentifier +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 SQLMetricsSuite extends SQLTestUtils with TestHiveSingleton { + import spark.implicits._ + + /** + * Get execution metrics for the SQL execution and verify metrics values. + * + * @param metricsValues the expected metric values (numFiles, numPartitions, numOutputRows). + * @param func the function can produce execution id after running. + */ + private def verifyWriteDataMetrics(metricsValues: Seq[Int])(func: => Unit): Unit = { + val previousExecutionIds = spark.sharedState.listener.executionIdToData.keySet + // Run the given function to trigger query execution. + func + spark.sparkContext.listenerBus.waitUntilEmpty(10000) + val executionIds = + spark.sharedState.listener.executionIdToData.keySet.diff(previousExecutionIds) + assert(executionIds.size == 1) + val executionId = executionIds.head + + val executionData = spark.sharedState.listener.getExecution(executionId).get + val executedNode = executionData.physicalPlanGraph.nodes.head + + val metricsNames = Seq( + "number of written files", + "number of dynamic part", + "number of output rows") + + val metrics = spark.sharedState.listener.getExecutionMetrics(executionId) + + metricsNames.zip(metricsValues).foreach { case (metricsName, expected) => + val sqlMetric = executedNode.metrics.find(_.name == metricsName) + assert(sqlMetric.isDefined) + val accumulatorId = sqlMetric.get.accumulatorId + val metricValue = metrics(accumulatorId).replaceAll(",", "").toInt + assert(metricValue == expected) + } + + val totalNumBytesMetric = executedNode.metrics.find(_.name == "bytes of written output").get + val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "").toInt + assert(totalNumBytes > 0) + val writingTimeMetric = executedNode.metrics.find(_.name == "average writing time (ms)").get + val writingTime = metrics(writingTimeMetric.accumulatorId).replaceAll(",", "").toInt + assert(writingTime >= 0) + } + + private def testMetricsNonDynamicPartition( + dataFormat: String, + tableName: String): Unit = { + withTable(tableName) { + Seq((1, 2)).toDF("i", "j") + .write.format(dataFormat).mode("overwrite").saveAsTable(tableName) + + val tableLocation = + new File(spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)).location) + + // 2 files, 100 rows, 0 dynamic partition. + verifyWriteDataMetrics(Seq(2, 0, 100)) { + (0 until 100).map(i => (i, i + 1)).toDF("i", "j").repartition(2) + .write.format(dataFormat).mode("overwrite").insertInto(tableName) + } + assert(Utils.recursiveList(tableLocation).count(_.getName.startsWith("part-")) == 2) + } + } + + private def testMetricsDynamicPartition( + provider: String, + dataFormat: String, + tableName: String): Unit = { + withTempPath { dir => + spark.sql( + s""" + |CREATE TABLE $tableName(a int, b int) + |USING $provider + |PARTITIONED BY(a) + |LOCATION '${dir.toURI}' + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + val df = spark.range(start = 0, end = 40, step = 1, numPartitions = 1) + .selectExpr("id a", "id b") + + // 40 files, 80 rows, 40 dynamic partitions. + verifyWriteDataMetrics(Seq(40, 40, 80)) { + df.union(df).repartition(2, $"a") + .write + .format(dataFormat) + .mode("overwrite") + .insertInto(tableName) + } + assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 40) + } + } + + test("writing data out metrics: parquet") { + testMetricsNonDynamicPartition("parquet", "t1") + } + + test("writing data out metrics with dynamic partition: parquet") { + testMetricsDynamicPartition("parquet", "parquet", "t1") + } + + test("writing data out metrics: hive") { + testMetricsNonDynamicPartition("hive", "t1") + } + + test("writing data out metrics dynamic partition: hive") { + withSQLConf(("hive.exec.dynamic.partition.mode", "nonstrict")) { + testMetricsDynamicPartition("hive", "hive", "t1") + } + } +} From b8e4d567a7d6c2ff277700d4e7707e57e87c7808 Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Thu, 6 Jul 2017 16:00:31 +0800 Subject: [PATCH 053/125] [SPARK-21324][TEST] Improve statistics test suites ## What changes were proposed in this pull request? 1. move `StatisticsCollectionTestBase` to a separate file. 2. move some test cases to `StatisticsCollectionSuite` so that `hive/StatisticsSuite` only keeps tests that need hive support. 3. clear up some test cases. ## How was this patch tested? Existing tests. Author: wangzhenhua Author: Zhenhua Wang Closes #18545 from wzhfy/cleanStatSuites. --- .../spark/sql/StatisticsCollectionSuite.scala | 193 +++--------------- .../sql/StatisticsCollectionTestBase.scala | 192 +++++++++++++++++ .../spark/sql/hive/StatisticsSuite.scala | 124 +++-------- 3 files changed, 258 insertions(+), 251 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala index d9392de37a81..843ced7f0e69 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala @@ -17,19 +17,12 @@ package org.apache.spark.sql -import java.{lang => jl} -import java.sql.{Date, Timestamp} - import scala.collection.mutable -import scala.util.Random import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogStatistics} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.test.SQLTestData.ArrayData import org.apache.spark.sql.types._ @@ -58,6 +51,37 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared } } + test("analyzing views is not supported") { + def assertAnalyzeUnsupported(analyzeCommand: String): Unit = { + val err = intercept[AnalysisException] { + sql(analyzeCommand) + } + assert(err.message.contains("ANALYZE TABLE is not supported")) + } + + val tableName = "tbl" + withTable(tableName) { + spark.range(10).write.saveAsTable(tableName) + val viewName = "view" + withView(viewName) { + sql(s"CREATE VIEW $viewName AS SELECT * FROM $tableName") + assertAnalyzeUnsupported(s"ANALYZE TABLE $viewName COMPUTE STATISTICS") + assertAnalyzeUnsupported(s"ANALYZE TABLE $viewName COMPUTE STATISTICS FOR COLUMNS id") + } + } + } + + test("statistics collection of a table with zero column") { + val table_no_cols = "table_no_cols" + withTable(table_no_cols) { + val rddNoCols = sparkContext.parallelize(1 to 10).map(_ => Row.empty) + val dfNoCols = spark.createDataFrame(rddNoCols, StructType(Seq.empty)) + dfNoCols.write.format("json").saveAsTable(table_no_cols) + sql(s"ANALYZE TABLE $table_no_cols COMPUTE STATISTICS") + checkTableStats(table_no_cols, hasSizeInBytes = true, expectedRowCounts = Some(10)) + } + } + test("analyze column command - unsupported types and invalid columns") { val tableName = "column_stats_test1" withTable(tableName) { @@ -239,154 +263,3 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared } } - - -/** - * The base for test cases that we want to include in both the hive module (for verifying behavior - * when using the Hive external catalog) as well as in the sql/core module. - */ -abstract class StatisticsCollectionTestBase extends QueryTest with SQLTestUtils { - import testImplicits._ - - private val dec1 = new java.math.BigDecimal("1.000000000000000000") - private val dec2 = new java.math.BigDecimal("8.000000000000000000") - private val d1 = Date.valueOf("2016-05-08") - private val d2 = Date.valueOf("2016-05-09") - private val t1 = Timestamp.valueOf("2016-05-08 00:00:01") - private val t2 = Timestamp.valueOf("2016-05-09 00:00:02") - - /** - * Define a very simple 3 row table used for testing column serialization. - * Note: last column is seq[int] which doesn't support stats collection. - */ - protected val data = Seq[ - (jl.Boolean, jl.Byte, jl.Short, jl.Integer, jl.Long, - jl.Double, jl.Float, java.math.BigDecimal, - String, Array[Byte], Date, Timestamp, - Seq[Int])]( - (false, 1.toByte, 1.toShort, 1, 1L, 1.0, 1.0f, dec1, "s1", "b1".getBytes, d1, t1, null), - (true, 2.toByte, 3.toShort, 4, 5L, 6.0, 7.0f, dec2, "ss9", "bb0".getBytes, d2, t2, null), - (null, null, null, null, null, null, null, null, null, null, null, null, null) - ) - - /** A mapping from column to the stats collected. */ - protected val stats = mutable.LinkedHashMap( - "cbool" -> ColumnStat(2, Some(false), Some(true), 1, 1, 1), - "cbyte" -> ColumnStat(2, Some(1.toByte), Some(2.toByte), 1, 1, 1), - "cshort" -> ColumnStat(2, Some(1.toShort), Some(3.toShort), 1, 2, 2), - "cint" -> ColumnStat(2, Some(1), Some(4), 1, 4, 4), - "clong" -> ColumnStat(2, Some(1L), Some(5L), 1, 8, 8), - "cdouble" -> ColumnStat(2, Some(1.0), Some(6.0), 1, 8, 8), - "cfloat" -> ColumnStat(2, Some(1.0f), Some(7.0f), 1, 4, 4), - "cdecimal" -> ColumnStat(2, Some(Decimal(dec1)), Some(Decimal(dec2)), 1, 16, 16), - "cstring" -> ColumnStat(2, None, None, 1, 3, 3), - "cbinary" -> ColumnStat(2, None, None, 1, 3, 3), - "cdate" -> ColumnStat(2, Some(DateTimeUtils.fromJavaDate(d1)), - Some(DateTimeUtils.fromJavaDate(d2)), 1, 4, 4), - "ctimestamp" -> ColumnStat(2, Some(DateTimeUtils.fromJavaTimestamp(t1)), - Some(DateTimeUtils.fromJavaTimestamp(t2)), 1, 8, 8) - ) - - private val randomName = new Random(31) - - def checkTableStats( - tableName: String, - hasSizeInBytes: Boolean, - expectedRowCounts: Option[Int]): Option[CatalogStatistics] = { - val stats = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)).stats - if (hasSizeInBytes || expectedRowCounts.nonEmpty) { - assert(stats.isDefined) - assert(stats.get.sizeInBytes >= 0) - assert(stats.get.rowCount === expectedRowCounts) - } else { - assert(stats.isEmpty) - } - - stats - } - - /** - * Compute column stats for the given DataFrame and compare it with colStats. - */ - def checkColStats( - df: DataFrame, - colStats: mutable.LinkedHashMap[String, ColumnStat]): Unit = { - val tableName = "column_stats_test_" + randomName.nextInt(1000) - withTable(tableName) { - df.write.saveAsTable(tableName) - - // Collect statistics - sql(s"analyze table $tableName compute STATISTICS FOR COLUMNS " + - colStats.keys.mkString(", ")) - - // Validate statistics - val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) - assert(table.stats.isDefined) - assert(table.stats.get.colStats.size == colStats.size) - - colStats.foreach { case (k, v) => - withClue(s"column $k") { - assert(table.stats.get.colStats(k) == v) - } - } - } - } - - // This test will be run twice: with and without Hive support - test("SPARK-18856: non-empty partitioned table should not report zero size") { - withTable("ds_tbl", "hive_tbl") { - spark.range(100).select($"id", $"id" % 5 as "p").write.partitionBy("p").saveAsTable("ds_tbl") - val stats = spark.table("ds_tbl").queryExecution.optimizedPlan.stats - assert(stats.sizeInBytes > 0, "non-empty partitioned table should not report zero size.") - - if (spark.conf.get(StaticSQLConf.CATALOG_IMPLEMENTATION) == "hive") { - sql("CREATE TABLE hive_tbl(i int) PARTITIONED BY (j int)") - sql("INSERT INTO hive_tbl PARTITION(j=1) SELECT 1") - val stats2 = spark.table("hive_tbl").queryExecution.optimizedPlan.stats - assert(stats2.sizeInBytes > 0, "non-empty partitioned table should not report zero size.") - } - } - } - - // This test will be run twice: with and without Hive support - test("conversion from CatalogStatistics to Statistics") { - withTable("ds_tbl", "hive_tbl") { - // Test data source table - checkStatsConversion(tableName = "ds_tbl", isDatasourceTable = true) - // Test hive serde table - if (spark.conf.get(StaticSQLConf.CATALOG_IMPLEMENTATION) == "hive") { - checkStatsConversion(tableName = "hive_tbl", isDatasourceTable = false) - } - } - } - - private def checkStatsConversion(tableName: String, isDatasourceTable: Boolean): Unit = { - // Create an empty table and run analyze command on it. - val createTableSql = if (isDatasourceTable) { - s"CREATE TABLE $tableName (c1 INT, c2 STRING) USING PARQUET" - } else { - s"CREATE TABLE $tableName (c1 INT, c2 STRING)" - } - sql(createTableSql) - // Analyze only one column. - sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS c1") - val (relation, catalogTable) = spark.table(tableName).queryExecution.analyzed.collect { - case catalogRel: CatalogRelation => (catalogRel, catalogRel.tableMeta) - case logicalRel: LogicalRelation => (logicalRel, logicalRel.catalogTable.get) - }.head - val emptyColStat = ColumnStat(0, None, None, 0, 4, 4) - // Check catalog statistics - assert(catalogTable.stats.isDefined) - assert(catalogTable.stats.get.sizeInBytes == 0) - assert(catalogTable.stats.get.rowCount == Some(0)) - assert(catalogTable.stats.get.colStats == Map("c1" -> emptyColStat)) - - // Check relation statistics - assert(relation.stats.sizeInBytes == 0) - assert(relation.stats.rowCount == Some(0)) - assert(relation.stats.attributeStats.size == 1) - val (attribute, colStat) = relation.stats.attributeStats.head - assert(attribute.name == "c1") - assert(colStat == emptyColStat) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala new file mode 100644 index 000000000000..41569762d3c5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.{lang => jl} +import java.sql.{Date, Timestamp} + +import scala.collection.mutable +import scala.util.Random + +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogStatistics, CatalogTable} +import org.apache.spark.sql.catalyst.plans.logical.ColumnStat +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.internal.StaticSQLConf +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.Decimal + + +/** + * The base for statistics test cases that we want to include in both the hive module (for + * verifying behavior when using the Hive external catalog) as well as in the sql/core module. + */ +abstract class StatisticsCollectionTestBase extends QueryTest with SQLTestUtils { + import testImplicits._ + + private val dec1 = new java.math.BigDecimal("1.000000000000000000") + private val dec2 = new java.math.BigDecimal("8.000000000000000000") + private val d1 = Date.valueOf("2016-05-08") + private val d2 = Date.valueOf("2016-05-09") + private val t1 = Timestamp.valueOf("2016-05-08 00:00:01") + private val t2 = Timestamp.valueOf("2016-05-09 00:00:02") + + /** + * Define a very simple 3 row table used for testing column serialization. + * Note: last column is seq[int] which doesn't support stats collection. + */ + protected val data = Seq[ + (jl.Boolean, jl.Byte, jl.Short, jl.Integer, jl.Long, + jl.Double, jl.Float, java.math.BigDecimal, + String, Array[Byte], Date, Timestamp, + Seq[Int])]( + (false, 1.toByte, 1.toShort, 1, 1L, 1.0, 1.0f, dec1, "s1", "b1".getBytes, d1, t1, null), + (true, 2.toByte, 3.toShort, 4, 5L, 6.0, 7.0f, dec2, "ss9", "bb0".getBytes, d2, t2, null), + (null, null, null, null, null, null, null, null, null, null, null, null, null) + ) + + /** A mapping from column to the stats collected. */ + protected val stats = mutable.LinkedHashMap( + "cbool" -> ColumnStat(2, Some(false), Some(true), 1, 1, 1), + "cbyte" -> ColumnStat(2, Some(1.toByte), Some(2.toByte), 1, 1, 1), + "cshort" -> ColumnStat(2, Some(1.toShort), Some(3.toShort), 1, 2, 2), + "cint" -> ColumnStat(2, Some(1), Some(4), 1, 4, 4), + "clong" -> ColumnStat(2, Some(1L), Some(5L), 1, 8, 8), + "cdouble" -> ColumnStat(2, Some(1.0), Some(6.0), 1, 8, 8), + "cfloat" -> ColumnStat(2, Some(1.0f), Some(7.0f), 1, 4, 4), + "cdecimal" -> ColumnStat(2, Some(Decimal(dec1)), Some(Decimal(dec2)), 1, 16, 16), + "cstring" -> ColumnStat(2, None, None, 1, 3, 3), + "cbinary" -> ColumnStat(2, None, None, 1, 3, 3), + "cdate" -> ColumnStat(2, Some(DateTimeUtils.fromJavaDate(d1)), + Some(DateTimeUtils.fromJavaDate(d2)), 1, 4, 4), + "ctimestamp" -> ColumnStat(2, Some(DateTimeUtils.fromJavaTimestamp(t1)), + Some(DateTimeUtils.fromJavaTimestamp(t2)), 1, 8, 8) + ) + + private val randomName = new Random(31) + + def getCatalogTable(tableName: String): CatalogTable = { + spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) + } + + def getCatalogStatistics(tableName: String): CatalogStatistics = { + getCatalogTable(tableName).stats.get + } + + def checkTableStats( + tableName: String, + hasSizeInBytes: Boolean, + expectedRowCounts: Option[Int]): Option[CatalogStatistics] = { + val stats = getCatalogTable(tableName).stats + if (hasSizeInBytes || expectedRowCounts.nonEmpty) { + assert(stats.isDefined) + assert(stats.get.sizeInBytes >= 0) + assert(stats.get.rowCount === expectedRowCounts) + } else { + assert(stats.isEmpty) + } + + stats + } + + /** + * Compute column stats for the given DataFrame and compare it with colStats. + */ + def checkColStats( + df: DataFrame, + colStats: mutable.LinkedHashMap[String, ColumnStat]): Unit = { + val tableName = "column_stats_test_" + randomName.nextInt(1000) + withTable(tableName) { + df.write.saveAsTable(tableName) + + // Collect statistics + sql(s"analyze table $tableName compute STATISTICS FOR COLUMNS " + + colStats.keys.mkString(", ")) + + // Validate statistics + val table = getCatalogTable(tableName) + assert(table.stats.isDefined) + assert(table.stats.get.colStats.size == colStats.size) + + colStats.foreach { case (k, v) => + withClue(s"column $k") { + assert(table.stats.get.colStats(k) == v) + } + } + } + } + + // This test will be run twice: with and without Hive support + test("SPARK-18856: non-empty partitioned table should not report zero size") { + withTable("ds_tbl", "hive_tbl") { + spark.range(100).select($"id", $"id" % 5 as "p").write.partitionBy("p").saveAsTable("ds_tbl") + val stats = spark.table("ds_tbl").queryExecution.optimizedPlan.stats + assert(stats.sizeInBytes > 0, "non-empty partitioned table should not report zero size.") + + if (spark.conf.get(StaticSQLConf.CATALOG_IMPLEMENTATION) == "hive") { + sql("CREATE TABLE hive_tbl(i int) PARTITIONED BY (j int)") + sql("INSERT INTO hive_tbl PARTITION(j=1) SELECT 1") + val stats2 = spark.table("hive_tbl").queryExecution.optimizedPlan.stats + assert(stats2.sizeInBytes > 0, "non-empty partitioned table should not report zero size.") + } + } + } + + // This test will be run twice: with and without Hive support + test("conversion from CatalogStatistics to Statistics") { + withTable("ds_tbl", "hive_tbl") { + // Test data source table + checkStatsConversion(tableName = "ds_tbl", isDatasourceTable = true) + // Test hive serde table + if (spark.conf.get(StaticSQLConf.CATALOG_IMPLEMENTATION) == "hive") { + checkStatsConversion(tableName = "hive_tbl", isDatasourceTable = false) + } + } + } + + private def checkStatsConversion(tableName: String, isDatasourceTable: Boolean): Unit = { + // Create an empty table and run analyze command on it. + val createTableSql = if (isDatasourceTable) { + s"CREATE TABLE $tableName (c1 INT, c2 STRING) USING PARQUET" + } else { + s"CREATE TABLE $tableName (c1 INT, c2 STRING)" + } + sql(createTableSql) + // Analyze only one column. + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS c1") + val (relation, catalogTable) = spark.table(tableName).queryExecution.analyzed.collect { + case catalogRel: CatalogRelation => (catalogRel, catalogRel.tableMeta) + case logicalRel: LogicalRelation => (logicalRel, logicalRel.catalogTable.get) + }.head + val emptyColStat = ColumnStat(0, None, None, 0, 4, 4) + // Check catalog statistics + assert(catalogTable.stats.isDefined) + assert(catalogTable.stats.get.sizeInBytes == 0) + assert(catalogTable.stats.get.rowCount == Some(0)) + assert(catalogTable.stats.get.colStats == Map("c1" -> emptyColStat)) + + // Check relation statistics + assert(relation.stats.sizeInBytes == 0) + assert(relation.stats.rowCount == Some(0)) + assert(relation.stats.attributeStats.size == 1) + val (attribute, colStat) = relation.stats.attributeStats.head + assert(attribute.name == "c1") + assert(colStat == emptyColStat) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index c601038a2b0a..e00fa64e9f2c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -25,7 +25,7 @@ import scala.util.matching.Regex import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogStatistics, CatalogTable} +import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogStatistics} import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.LogicalRelation @@ -33,7 +33,6 @@ import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.hive.HiveExternalCatalog._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types._ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleton { @@ -82,58 +81,42 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto spark.table(tableName).queryExecution.analyzed.stats.sizeInBytes // Non-partitioned table - sql("CREATE TABLE analyzeTable (key STRING, value STRING)").collect() - sql("INSERT INTO TABLE analyzeTable SELECT * FROM src").collect() - sql("INSERT INTO TABLE analyzeTable SELECT * FROM src").collect() + val nonPartTable = "non_part_table" + withTable(nonPartTable) { + sql(s"CREATE TABLE $nonPartTable (key STRING, value STRING)") + sql(s"INSERT INTO TABLE $nonPartTable SELECT * FROM src") + sql(s"INSERT INTO TABLE $nonPartTable SELECT * FROM src") - sql("ANALYZE TABLE analyzeTable COMPUTE STATISTICS noscan") + sql(s"ANALYZE TABLE $nonPartTable COMPUTE STATISTICS noscan") - assert(queryTotalSize("analyzeTable") === BigInt(11624)) - - sql("DROP TABLE analyzeTable").collect() + assert(queryTotalSize(nonPartTable) === BigInt(11624)) + } // Partitioned table - sql( - """ - |CREATE TABLE analyzeTable_part (key STRING, value STRING) PARTITIONED BY (ds STRING) - """.stripMargin).collect() - sql( - """ - |INSERT INTO TABLE analyzeTable_part PARTITION (ds='2010-01-01') - |SELECT * FROM src - """.stripMargin).collect() - sql( - """ - |INSERT INTO TABLE analyzeTable_part PARTITION (ds='2010-01-02') - |SELECT * FROM src - """.stripMargin).collect() - sql( - """ - |INSERT INTO TABLE analyzeTable_part PARTITION (ds='2010-01-03') - |SELECT * FROM src - """.stripMargin).collect() + val partTable = "part_table" + withTable(partTable) { + sql(s"CREATE TABLE $partTable (key STRING, value STRING) PARTITIONED BY (ds STRING)") + sql(s"INSERT INTO TABLE $partTable PARTITION (ds='2010-01-01') SELECT * FROM src") + sql(s"INSERT INTO TABLE $partTable PARTITION (ds='2010-01-02') SELECT * FROM src") + sql(s"INSERT INTO TABLE $partTable PARTITION (ds='2010-01-03') SELECT * FROM src") - assert(queryTotalSize("analyzeTable_part") === spark.sessionState.conf.defaultSizeInBytes) + assert(queryTotalSize(partTable) === spark.sessionState.conf.defaultSizeInBytes) - sql("ANALYZE TABLE analyzeTable_part COMPUTE STATISTICS noscan") + sql(s"ANALYZE TABLE $partTable COMPUTE STATISTICS noscan") - assert(queryTotalSize("analyzeTable_part") === BigInt(17436)) - - sql("DROP TABLE analyzeTable_part").collect() + assert(queryTotalSize(partTable) === BigInt(17436)) + } // Try to analyze a temp table - sql("""SELECT * FROM src""").createOrReplaceTempView("tempTable") - intercept[AnalysisException] { - sql("ANALYZE TABLE tempTable COMPUTE STATISTICS") + withView("tempTable") { + sql("""SELECT * FROM src""").createOrReplaceTempView("tempTable") + intercept[AnalysisException] { + sql("ANALYZE TABLE tempTable COMPUTE STATISTICS") + } } - spark.sessionState.catalog.dropTable( - TableIdentifier("tempTable"), ignoreIfNotExists = true, purge = false) } test("SPARK-21079 - analyze table with location different than that of individual partitions") { - def queryTotalSize(tableName: String): BigInt = - spark.table(tableName).queryExecution.analyzed.stats.sizeInBytes - val tableName = "analyzeTable_part" withTable(tableName) { withTempPath { path => @@ -148,15 +131,12 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS noscan") - assert(queryTotalSize(tableName) === BigInt(17436)) + assert(getCatalogStatistics(tableName).sizeInBytes === BigInt(17436)) } } } test("SPARK-21079 - analyze partitioned table with only a subset of partitions visible") { - def queryTotalSize(tableName: String): BigInt = - spark.table(tableName).queryExecution.analyzed.stats.sizeInBytes - val sourceTableName = "analyzeTable_part" val tableName = "analyzeTable_part_vis" withTable(sourceTableName, tableName) { @@ -188,39 +168,19 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto // Register only one of the partitions found on disk val ds = partitionDates.head - sql(s"ALTER TABLE $tableName ADD PARTITION (ds='$ds')").collect() + sql(s"ALTER TABLE $tableName ADD PARTITION (ds='$ds')") // Analyze original table - expect 3 partitions sql(s"ANALYZE TABLE $sourceTableName COMPUTE STATISTICS noscan") - assert(queryTotalSize(sourceTableName) === BigInt(3 * 5812)) + assert(getCatalogStatistics(sourceTableName).sizeInBytes === BigInt(3 * 5812)) // Analyze partial-copy table - expect only 1 partition sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS noscan") - assert(queryTotalSize(tableName) === BigInt(5812)) + assert(getCatalogStatistics(tableName).sizeInBytes === BigInt(5812)) } } } - test("analyzing views is not supported") { - def assertAnalyzeUnsupported(analyzeCommand: String): Unit = { - val err = intercept[AnalysisException] { - sql(analyzeCommand) - } - assert(err.message.contains("ANALYZE TABLE is not supported")) - } - - val tableName = "tbl" - withTable(tableName) { - spark.range(10).write.saveAsTable(tableName) - val viewName = "view" - withView(viewName) { - sql(s"CREATE VIEW $viewName AS SELECT * FROM $tableName") - assertAnalyzeUnsupported(s"ANALYZE TABLE $viewName COMPUTE STATISTICS") - assertAnalyzeUnsupported(s"ANALYZE TABLE $viewName COMPUTE STATISTICS FOR COLUMNS id") - } - } - } - test("test table-level statistics for hive tables created in HiveExternalCatalog") { val textTable = "textTable" withTable(textTable) { @@ -290,8 +250,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto if (analyzedByHive) hiveClient.runSqlHive(s"ANALYZE TABLE $tabName COMPUTE STATISTICS") val describeResult1 = hiveClient.runSqlHive(s"DESCRIBE FORMATTED $tabName") - val tableMetadata = - spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName)).properties + val tableMetadata = getCatalogTable(tabName).properties // statistics info is not contained in the metadata of the original table assert(Seq(StatsSetupConst.COLUMN_STATS_ACCURATE, StatsSetupConst.NUM_FILES, @@ -327,8 +286,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto val tabName = "tab1" withTable(tabName) { createNonPartitionedTable(tabName, analyzedByHive = false, analyzedBySpark = false) - checkTableStats( - tabName, hasSizeInBytes = true, expectedRowCounts = None) + checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = None) // ALTER TABLE SET TBLPROPERTIES invalidates some contents of Hive specific statistics // This is triggered by the Hive alterTable API @@ -370,10 +328,6 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto } test("alter table should not have the side effect to store statistics in Spark side") { - def getCatalogTable(tableName: String): CatalogTable = { - spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) - } - val table = "alter_table_side_effect" withTable(table) { sql(s"CREATE TABLE $table (i string, j string)") @@ -637,12 +591,12 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto // the default value for `spark.sql.hive.convertMetastoreParquet` is true, here we just set it // for robustness - withSQLConf("spark.sql.hive.convertMetastoreParquet" -> "true") { + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "true") { checkTableStats(parquetTable, hasSizeInBytes = false, expectedRowCounts = None) sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") checkTableStats(parquetTable, hasSizeInBytes = true, expectedRowCounts = Some(500)) } - withSQLConf("spark.sql.hive.convertMetastoreOrc" -> "true") { + withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "true") { // We still can get tableSize from Hive before Analyze checkTableStats(orcTable, hasSizeInBytes = true, expectedRowCounts = None) sql(s"ANALYZE TABLE $orcTable COMPUTE STATISTICS") @@ -759,8 +713,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto val parquetTable = "parquetTable" withTable(parquetTable) { sql(createTableCmd) - val catalogTable = spark.sessionState.catalog.getTableMetadata( - TableIdentifier(parquetTable)) + val catalogTable = getCatalogTable(parquetTable) assert(DDLUtils.isDatasourceTable(catalogTable)) // Add a filter to avoid creating too many partitions @@ -795,17 +748,6 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto "partitioned data source table", "CREATE TABLE parquetTable (key STRING, value STRING) USING PARQUET PARTITIONED BY (key)") - test("statistics collection of a table with zero column") { - val table_no_cols = "table_no_cols" - withTable(table_no_cols) { - val rddNoCols = sparkContext.parallelize(1 to 10).map(_ => Row.empty) - val dfNoCols = spark.createDataFrame(rddNoCols, StructType(Seq.empty)) - dfNoCols.write.format("json").saveAsTable(table_no_cols) - sql(s"ANALYZE TABLE $table_no_cols COMPUTE STATISTICS") - checkTableStats(table_no_cols, hasSizeInBytes = true, expectedRowCounts = Some(10)) - } - } - /** Used to test refreshing cached metadata once table stats are updated. */ private def getStatsBeforeAfterUpdate(isAnalyzeColumns: Boolean) : (CatalogStatistics, CatalogStatistics) = { From d540dfbff33aa2f8571e0de149dfa3f4e7321113 Mon Sep 17 00:00:00 2001 From: Wang Gengliang Date: Thu, 6 Jul 2017 19:12:15 +0800 Subject: [PATCH 054/125] [SPARK-21273][SQL][FOLLOW-UP] Add missing test cases back and revise code style ## What changes were proposed in this pull request? Add missing test cases back and revise code style Follow up the previous PR: https://github.com/apache/spark/pull/18479 ## How was this patch tested? Unit test Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Wang Gengliang Closes #18548 from gengliangwang/stat_propagation_revise. --- .../plans/logical/LogicalPlanVisitor.scala | 2 +- .../BasicStatsEstimationSuite.scala | 45 +++++++++++++++++++ 2 files changed, 46 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala index b23045810a4f..2652f6d72730 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala @@ -38,10 +38,10 @@ trait LogicalPlanVisitor[T] { case p: Range => visitRange(p) case p: Repartition => visitRepartition(p) case p: RepartitionByExpression => visitRepartitionByExpr(p) + case p: ResolvedHint => visitHint(p) case p: Sample => visitSample(p) case p: ScriptTransformation => visitScriptTransform(p) case p: Union => visitUnion(p) - case p: ResolvedHint => visitHint(p) case p: LogicalPlan => default(p) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala index 5fd21a06a109..913be6d1ff07 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala @@ -78,6 +78,37 @@ class BasicStatsEstimationSuite extends PlanTest with StatsEstimationTestBase { checkStats(globalLimit, stats) } + test("sample estimation") { + val sample = Sample(0.0, 0.5, withReplacement = false, (math.random * 1000).toLong, plan) + checkStats(sample, Statistics(sizeInBytes = 60, rowCount = Some(5))) + + // Child doesn't have rowCount in stats + val childStats = Statistics(sizeInBytes = 120) + val childPlan = DummyLogicalPlan(childStats, childStats) + val sample2 = + Sample(0.0, 0.11, withReplacement = false, (math.random * 1000).toLong, childPlan) + checkStats(sample2, Statistics(sizeInBytes = 14)) + } + + test("estimate statistics when the conf changes") { + val expectedDefaultStats = + Statistics( + sizeInBytes = 40, + rowCount = Some(10), + attributeStats = AttributeMap(Seq( + AttributeReference("c1", IntegerType)() -> ColumnStat(10, Some(1), Some(10), 0, 4, 4)))) + val expectedCboStats = + Statistics( + sizeInBytes = 4, + rowCount = Some(1), + attributeStats = AttributeMap(Seq( + AttributeReference("c1", IntegerType)() -> ColumnStat(1, Some(5), Some(5), 0, 4, 4)))) + + val plan = DummyLogicalPlan(defaultStats = expectedDefaultStats, cboStats = expectedCboStats) + checkStats( + plan, expectedStatsCboOn = expectedCboStats, expectedStatsCboOff = expectedDefaultStats) + } + /** Check estimated stats when cbo is turned on/off. */ private def checkStats( plan: LogicalPlan, @@ -99,3 +130,17 @@ class BasicStatsEstimationSuite extends PlanTest with StatsEstimationTestBase { private def checkStats(plan: LogicalPlan, expectedStats: Statistics): Unit = checkStats(plan, expectedStats, expectedStats) } + +/** + * This class is used for unit-testing the cbo switch, it mimics a logical plan which computes + * a simple statistics or a cbo estimated statistics based on the conf. + */ +private case class DummyLogicalPlan( + defaultStats: Statistics, + cboStats: Statistics) + extends LeafNode { + + override def output: Seq[Attribute] = Nil + + override def computeStats(): Statistics = if (conf.cboEnabled) cboStats else defaultStats +} From 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e Mon Sep 17 00:00:00 2001 From: caoxuewen Date: Thu, 6 Jul 2017 19:49:34 +0800 Subject: [PATCH 055/125] [SPARK-20950][CORE] add a new config to diskWriteBufferSize which is hard coded before ## What changes were proposed in this pull request? This PR Improvement in two: 1.With spark.shuffle.spill.diskWriteBufferSize configure diskWriteBufferSize of ShuffleExternalSorter. when change the size of the diskWriteBufferSize to test `forceSorterToSpill` The average performance of running 10 times is as follows:(their unit is MS). ``` diskWriteBufferSize: 1M 512K 256K 128K 64K 32K 16K 8K 4K --------------------------------------------------------------------------------------- RecordSize = 2.5M 742 722 694 686 667 668 671 669 683 RecordSize = 1M 294 293 292 287 283 285 281 279 285 ``` 2.Remove outputBufferSizeInBytes and inputBufferSizeInBytes to initialize in mergeSpillsWithFileStream function. ## How was this patch tested? The unit test. Author: caoxuewen Closes #18174 from heary-cao/buffersize. --- .../shuffle/sort/ShuffleExternalSorter.java | 11 +++++--- .../shuffle/sort/UnsafeShuffleWriter.java | 14 +++++++--- .../unsafe/sort/UnsafeSorterSpillWriter.java | 24 ++++++++++------- .../spark/internal/config/package.scala | 27 +++++++++++++++++++ 4 files changed, 60 insertions(+), 16 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index c33d1e33f030..338faaadb33d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -43,6 +43,7 @@ import org.apache.spark.unsafe.array.LongArray; import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.util.Utils; +import org.apache.spark.internal.config.package$; /** * An external sorter that is specialized for sort-based shuffle. @@ -82,6 +83,9 @@ final class ShuffleExternalSorter extends MemoryConsumer { /** The buffer size to use when writing spills using DiskBlockObjectWriter */ private final int fileBufferSizeBytes; + /** The buffer size to use when writing the sorted records to an on-disk file */ + private final int diskWriteBufferSize; + /** * Memory pages that hold the records being sorted. The pages in this list are freed when * spilling, although in principle we could recycle these pages across spills (on the other hand, @@ -116,13 +120,14 @@ final class ShuffleExternalSorter extends MemoryConsumer { this.taskContext = taskContext; this.numPartitions = numPartitions; // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided - this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024; + this.fileBufferSizeBytes = (int) (long) conf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; this.numElementsForSpillThreshold = conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", 1024 * 1024 * 1024); this.writeMetrics = writeMetrics; this.inMemSorter = new ShuffleInMemorySorter( this, initialSize, conf.getBoolean("spark.shuffle.sort.useRadixSort", true)); this.peakMemoryUsedBytes = getMemoryUsage(); + this.diskWriteBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE()); } /** @@ -155,7 +160,7 @@ private void writeSortedFile(boolean isLastFile) throws IOException { // be an API to directly transfer bytes from managed memory to the disk writer, we buffer // data through a byte array. This array does not need to be large enough to hold a single // record; - final byte[] writeBuffer = new byte[DISK_WRITE_BUFFER_SIZE]; + final byte[] writeBuffer = new byte[diskWriteBufferSize]; // Because this output will be read during shuffle, its compression codec must be controlled by // spark.shuffle.compress instead of spark.shuffle.spill.compress, so we need to use @@ -195,7 +200,7 @@ private void writeSortedFile(boolean isLastFile) throws IOException { int dataRemaining = Platform.getInt(recordPage, recordOffsetInPage); long recordReadPosition = recordOffsetInPage + 4; // skip over record length while (dataRemaining > 0) { - final int toTransfer = Math.min(DISK_WRITE_BUFFER_SIZE, dataRemaining); + final int toTransfer = Math.min(diskWriteBufferSize, dataRemaining); Platform.copyMemory( recordPage, recordReadPosition, writeBuffer, Platform.BYTE_ARRAY_OFFSET, toTransfer); writer.write(writeBuffer, 0, toTransfer); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 34c179990214..1b578491b81d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -55,6 +55,7 @@ import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; import org.apache.spark.util.Utils; +import org.apache.spark.internal.config.package$; @Private public class UnsafeShuffleWriter extends ShuffleWriter { @@ -65,6 +66,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { @VisibleForTesting static final int DEFAULT_INITIAL_SORT_BUFFER_SIZE = 4096; + static final int DEFAULT_INITIAL_SER_BUFFER_SIZE = 1024 * 1024; private final BlockManager blockManager; private final IndexShuffleBlockResolver shuffleBlockResolver; @@ -78,6 +80,8 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final SparkConf sparkConf; private final boolean transferToEnabled; private final int initialSortBufferSize; + private final int inputBufferSizeInBytes; + private final int outputBufferSizeInBytes; @Nullable private MapStatus mapStatus; @Nullable private ShuffleExternalSorter sorter; @@ -140,6 +144,10 @@ public UnsafeShuffleWriter( this.transferToEnabled = sparkConf.getBoolean("spark.file.transferTo", true); this.initialSortBufferSize = sparkConf.getInt("spark.shuffle.sort.initialBufferSize", DEFAULT_INITIAL_SORT_BUFFER_SIZE); + this.inputBufferSizeInBytes = + (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; + this.outputBufferSizeInBytes = + (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE()) * 1024; open(); } @@ -209,7 +217,7 @@ private void open() throws IOException { partitioner.numPartitions(), sparkConf, writeMetrics); - serBuffer = new MyByteArrayOutputStream(1024 * 1024); + serBuffer = new MyByteArrayOutputStream(DEFAULT_INITIAL_SER_BUFFER_SIZE); serOutputStream = serializer.serializeStream(serBuffer); } @@ -360,12 +368,10 @@ private long[] mergeSpillsWithFileStream( final OutputStream bos = new BufferedOutputStream( new FileOutputStream(outputFile), - (int) sparkConf.getSizeAsKb("spark.shuffle.unsafe.file.output.buffer", "32k") * 1024); + outputBufferSizeInBytes); // Use a counting output stream to avoid having to close the underlying file and ask // the file system for its size after each partition is written. final CountingOutputStream mergedFileOutputStream = new CountingOutputStream(bos); - final int inputBufferSizeInBytes = - (int) sparkConf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024; boolean threwException = true; try { diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java index 164b9d70b79d..f9b549375544 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java @@ -20,9 +20,10 @@ import java.io.File; import java.io.IOException; -import org.apache.spark.serializer.SerializerManager; import scala.Tuple2; +import org.apache.spark.SparkConf; +import org.apache.spark.serializer.SerializerManager; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.serializer.DummySerializerInstance; import org.apache.spark.storage.BlockId; @@ -30,6 +31,7 @@ import org.apache.spark.storage.DiskBlockObjectWriter; import org.apache.spark.storage.TempLocalBlockId; import org.apache.spark.unsafe.Platform; +import org.apache.spark.internal.config.package$; /** * Spills a list of sorted records to disk. Spill files have the following format: @@ -38,12 +40,16 @@ */ public final class UnsafeSorterSpillWriter { - static final int DISK_WRITE_BUFFER_SIZE = 1024 * 1024; + private final SparkConf conf = new SparkConf(); + + /** The buffer size to use when writing the sorted records to an on-disk file */ + private final int diskWriteBufferSize = + (int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE()); // Small writes to DiskBlockObjectWriter will be fairly inefficient. Since there doesn't seem to // be an API to directly transfer bytes from managed memory to the disk writer, we buffer // data through a byte array. - private byte[] writeBuffer = new byte[DISK_WRITE_BUFFER_SIZE]; + private byte[] writeBuffer = new byte[diskWriteBufferSize]; private final File file; private final BlockId blockId; @@ -114,7 +120,7 @@ public void write( writeIntToBuffer(recordLength, 0); writeLongToBuffer(keyPrefix, 4); int dataRemaining = recordLength; - int freeSpaceInWriteBuffer = DISK_WRITE_BUFFER_SIZE - 4 - 8; // space used by prefix + len + int freeSpaceInWriteBuffer = diskWriteBufferSize - 4 - 8; // space used by prefix + len long recordReadPosition = baseOffset; while (dataRemaining > 0) { final int toTransfer = Math.min(freeSpaceInWriteBuffer, dataRemaining); @@ -122,15 +128,15 @@ public void write( baseObject, recordReadPosition, writeBuffer, - Platform.BYTE_ARRAY_OFFSET + (DISK_WRITE_BUFFER_SIZE - freeSpaceInWriteBuffer), + Platform.BYTE_ARRAY_OFFSET + (diskWriteBufferSize - freeSpaceInWriteBuffer), toTransfer); - writer.write(writeBuffer, 0, (DISK_WRITE_BUFFER_SIZE - freeSpaceInWriteBuffer) + toTransfer); + writer.write(writeBuffer, 0, (diskWriteBufferSize - freeSpaceInWriteBuffer) + toTransfer); recordReadPosition += toTransfer; dataRemaining -= toTransfer; - freeSpaceInWriteBuffer = DISK_WRITE_BUFFER_SIZE; + freeSpaceInWriteBuffer = diskWriteBufferSize; } - if (freeSpaceInWriteBuffer < DISK_WRITE_BUFFER_SIZE) { - writer.write(writeBuffer, 0, (DISK_WRITE_BUFFER_SIZE - freeSpaceInWriteBuffer)); + if (freeSpaceInWriteBuffer < diskWriteBufferSize) { + writer.write(writeBuffer, 0, (diskWriteBufferSize - freeSpaceInWriteBuffer)); } writer.recordWritten(); } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 8dee0d970c4c..a629810bf093 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -336,4 +336,31 @@ package object config { "spark.") .booleanConf .createWithDefault(false) + + private[spark] val SHUFFLE_FILE_BUFFER_SIZE = + ConfigBuilder("spark.shuffle.file.buffer") + .doc("Size of the in-memory buffer for each shuffle file output stream. " + + "These buffers reduce the number of disk seeks and system calls made " + + "in creating intermediate shuffle files.") + .bytesConf(ByteUnit.KiB) + .checkValue(v => v > 0 && v <= Int.MaxValue / 1024, + s"The file buffer size must be greater than 0 and less than ${Int.MaxValue / 1024}.") + .createWithDefaultString("32k") + + private[spark] val SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE = + ConfigBuilder("spark.shuffle.unsafe.file.output.buffer") + .doc("The file system for this buffer size after each partition " + + "is written in unsafe shuffle writer.") + .bytesConf(ByteUnit.KiB) + .checkValue(v => v > 0 && v <= Int.MaxValue / 1024, + s"The buffer size must be greater than 0 and less than ${Int.MaxValue / 1024}.") + .createWithDefaultString("32k") + + private[spark] val SHUFFLE_DISK_WRITE_BUFFER_SIZE = + ConfigBuilder("spark.shuffle.spill.diskWriteBufferSize") + .doc("The buffer size to use when writing the sorted records to an on-disk file.") + .bytesConf(ByteUnit.BYTE) + .checkValue(v => v > 0 && v <= Int.MaxValue, + s"The buffer size must be greater than 0 and less than ${Int.MaxValue}.") + .createWithDefault(1024 * 1024) } From 26ac085debb54d0104762d1cd4187cdf73f301ba Mon Sep 17 00:00:00 2001 From: Bogdan Raducanu Date: Fri, 7 Jul 2017 01:04:57 +0800 Subject: [PATCH 056/125] [SPARK-21228][SQL] InSet incorrect handling of structs ## What changes were proposed in this pull request? When data type is struct, InSet now uses TypeUtils.getInterpretedOrdering (similar to EqualTo) to build a TreeSet. In other cases it will use a HashSet as before (which should be faster). Similarly, In.eval uses Ordering.equiv instead of equals. ## How was this patch tested? New test in SQLQuerySuite. Author: Bogdan Raducanu Closes #18455 from bogdanrdc/SPARK-21228. --- .../sql/catalyst/expressions/predicates.scala | 57 ++++++++++++------- .../catalyst/expressions/PredicateSuite.scala | 31 +++++----- .../catalyst/optimizer/OptimizeInSuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 22 +++++++ 4 files changed, 78 insertions(+), 34 deletions(-) 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 f3fe58caa6fe..7bf10f199f1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.catalyst.expressions +import scala.collection.immutable.TreeSet + import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} -import org.apache.spark.sql.catalyst.expressions.codegen.{Predicate => BasePredicate} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, GenerateSafeProjection, GenerateUnsafeProjection, Predicate => BasePredicate} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @@ -175,20 +176,23 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { |[${sub.output.map(_.dataType.catalogString).mkString(", ")}]. """.stripMargin) } else { - TypeCheckResult.TypeCheckSuccess + TypeUtils.checkForOrderingExpr(value.dataType, s"function $prettyName") } } case _ => - if (list.exists(l => l.dataType != value.dataType)) { - TypeCheckResult.TypeCheckFailure("Arguments must be same type") + val mismatchOpt = list.find(l => l.dataType != value.dataType) + if (mismatchOpt.isDefined) { + TypeCheckResult.TypeCheckFailure(s"Arguments must be same type but were: " + + s"${value.dataType} != ${mismatchOpt.get.dataType}") } else { - TypeCheckResult.TypeCheckSuccess + TypeUtils.checkForOrderingExpr(value.dataType, s"function $prettyName") } } } override def children: Seq[Expression] = value +: list lazy val inSetConvertible = list.forall(_.isInstanceOf[Literal]) + private lazy val ordering = TypeUtils.getInterpretedOrdering(value.dataType) override def nullable: Boolean = children.exists(_.nullable) override def foldable: Boolean = children.forall(_.foldable) @@ -203,10 +207,10 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { var hasNull = false list.foreach { e => val v = e.eval(input) - if (v == evaluatedValue) { - return true - } else if (v == null) { + if (v == null) { hasNull = true + } else if (ordering.equiv(v, evaluatedValue)) { + return true } } if (hasNull) { @@ -265,7 +269,7 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with override def nullable: Boolean = child.nullable || hasNull protected override def nullSafeEval(value: Any): Any = { - if (hset.contains(value)) { + if (set.contains(value)) { true } else if (hasNull) { null @@ -274,27 +278,40 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with } } - def getHSet(): Set[Any] = hset + @transient private[this] lazy val set = child.dataType match { + case _: AtomicType => hset + case _: NullType => hset + case _ => + // for structs use interpreted ordering to be able to compare UnsafeRows with non-UnsafeRows + TreeSet.empty(TypeUtils.getInterpretedOrdering(child.dataType)) ++ hset + } + + def getSet(): Set[Any] = set override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val setName = classOf[Set[Any]].getName val InSetName = classOf[InSet].getName val childGen = child.genCode(ctx) ctx.references += this - val hsetTerm = ctx.freshName("hset") - val hasNullTerm = ctx.freshName("hasNull") - ctx.addMutableState(setName, hsetTerm, - s"$hsetTerm = (($InSetName)references[${ctx.references.size - 1}]).getHSet();") - ctx.addMutableState("boolean", hasNullTerm, s"$hasNullTerm = $hsetTerm.contains(null);") + val setTerm = ctx.freshName("set") + val setNull = if (hasNull) { + s""" + |if (!${ev.value}) { + | ${ev.isNull} = true; + |} + """.stripMargin + } else { + "" + } + ctx.addMutableState(setName, setTerm, + s"$setTerm = (($InSetName)references[${ctx.references.size - 1}]).getSet();") ev.copy(code = s""" ${childGen.code} boolean ${ev.isNull} = ${childGen.isNull}; boolean ${ev.value} = false; if (!${ev.isNull}) { - ${ev.value} = $hsetTerm.contains(${childGen.value}); - if (!${ev.value} && $hasNullTerm) { - ${ev.isNull} = true; - } + ${ev.value} = $setTerm.contains(${childGen.value}); + $setNull } """) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala index 6fe295c3dd93..ef510a95ef44 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala @@ -35,7 +35,8 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { test(s"3VL $name") { truthTable.foreach { case (l, r, answer) => - val expr = op(NonFoldableLiteral(l, BooleanType), NonFoldableLiteral(r, BooleanType)) + val expr = op(NonFoldableLiteral.create(l, BooleanType), + NonFoldableLiteral.create(r, BooleanType)) checkEvaluation(expr, answer) } } @@ -72,7 +73,7 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { (false, true) :: (null, null) :: Nil notTrueTable.foreach { case (v, answer) => - checkEvaluation(Not(NonFoldableLiteral(v, BooleanType)), answer) + checkEvaluation(Not(NonFoldableLiteral.create(v, BooleanType)), answer) } checkConsistencyBetweenInterpretedAndCodegen(Not, BooleanType) } @@ -120,22 +121,26 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { (null, null, null) :: Nil) test("IN") { - checkEvaluation(In(NonFoldableLiteral(null, IntegerType), Seq(Literal(1), Literal(2))), null) - checkEvaluation(In(NonFoldableLiteral(null, IntegerType), - Seq(NonFoldableLiteral(null, IntegerType))), null) - checkEvaluation(In(NonFoldableLiteral(null, IntegerType), Seq.empty), null) + checkEvaluation(In(NonFoldableLiteral.create(null, IntegerType), Seq(Literal(1), + Literal(2))), null) + checkEvaluation(In(NonFoldableLiteral.create(null, IntegerType), + Seq(NonFoldableLiteral.create(null, IntegerType))), null) + checkEvaluation(In(NonFoldableLiteral.create(null, IntegerType), Seq.empty), null) checkEvaluation(In(Literal(1), Seq.empty), false) - checkEvaluation(In(Literal(1), Seq(NonFoldableLiteral(null, IntegerType))), null) - checkEvaluation(In(Literal(1), Seq(Literal(1), NonFoldableLiteral(null, IntegerType))), true) - checkEvaluation(In(Literal(2), Seq(Literal(1), NonFoldableLiteral(null, IntegerType))), null) + checkEvaluation(In(Literal(1), Seq(NonFoldableLiteral.create(null, IntegerType))), null) + checkEvaluation(In(Literal(1), Seq(Literal(1), NonFoldableLiteral.create(null, IntegerType))), + true) + checkEvaluation(In(Literal(2), Seq(Literal(1), NonFoldableLiteral.create(null, IntegerType))), + null) checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true) checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true) checkEvaluation(In(Literal(3), Seq(Literal(1), Literal(2))), false) checkEvaluation( - And(In(Literal(1), Seq(Literal(1), Literal(2))), In(Literal(2), Seq(Literal(1), Literal(2)))), + And(In(Literal(1), Seq(Literal(1), Literal(2))), In(Literal(2), Seq(Literal(1), + Literal(2)))), true) - val ns = NonFoldableLiteral(null, StringType) + val ns = NonFoldableLiteral.create(null, StringType) checkEvaluation(In(ns, Seq(Literal("1"), Literal("2"))), null) checkEvaluation(In(ns, Seq(ns)), null) checkEvaluation(In(Literal("a"), Seq(ns)), null) @@ -155,7 +160,7 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { case _ => value } } - val input = inputData.map(NonFoldableLiteral(_, t)) + val input = inputData.map(NonFoldableLiteral.create(_, t)) val expected = if (inputData(0) == null) { null } else if (inputData.slice(1, 10).contains(inputData(0))) { @@ -279,7 +284,7 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { test("BinaryComparison: null test") { // Use -1 (default value for codegen) which can trigger some weird bugs, e.g. SPARK-14757 val normalInt = Literal(-1) - val nullInt = NonFoldableLiteral(null, IntegerType) + val nullInt = NonFoldableLiteral.create(null, IntegerType) def nullTest(op: (Expression, Expression) => Expression): Unit = { checkEvaluation(op(normalInt, nullInt), null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala index 6a77580b29a2..28bf7b6f8434 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala @@ -169,7 +169,7 @@ class OptimizeInSuite extends PlanTest { val optimizedPlan = OptimizeIn(plan) optimizedPlan match { case Filter(cond, _) - if cond.isInstanceOf[InSet] && cond.asInstanceOf[InSet].getHSet().size == 3 => + if cond.isInstanceOf[InSet] && cond.asInstanceOf[InSet].getSet().size == 3 => // pass case _ => fail("Unexpected result for OptimizedIn") } 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 68f61cfab6d2..5171aaebc990 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 @@ -2616,4 +2616,26 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { val e = intercept[AnalysisException](sql("SELECT nvl(1, 2, 3)")) assert(e.message.contains("Invalid number of arguments")) } + + test("SPARK-21228: InSet incorrect handling of structs") { + withTempView("A") { + // reduce this from the default of 10 so the repro query text is not too long + withSQLConf((SQLConf.OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> "3")) { + // a relation that has 1 column of struct type with values (1,1), ..., (9, 9) + spark.range(1, 10).selectExpr("named_struct('a', id, 'b', id) as a") + .createOrReplaceTempView("A") + val df = sql( + """ + |SELECT * from + | (SELECT MIN(a) as minA FROM A) AA -- this Aggregate will return UnsafeRows + | -- the IN will become InSet with a Set of GenericInternalRows + | -- a GenericInternalRow is never equal to an UnsafeRow so the query would + | -- returns 0 results, which is incorrect + | WHERE minA IN (NAMED_STRUCT('a', 1L, 'b', 1L), NAMED_STRUCT('a', 2L, 'b', 2L), + | NAMED_STRUCT('a', 3L, 'b', 3L)) + """.stripMargin) + checkAnswer(df, Row(Row(1, 1))) + } + } + } } From 48e44b24a7663142176102ac4c6bf4242f103804 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 7 Jul 2017 01:07:45 +0800 Subject: [PATCH 057/125] [SPARK-21204][SQL] Add support for Scala Set collection types in serialization ## What changes were proposed in this pull request? Currently we can't produce a `Dataset` containing `Set` in SparkSQL. This PR tries to support serialization/deserialization of `Set`. Because there's no corresponding internal data type in SparkSQL for a `Set`, the most proper choice for serializing a set should be an array. ## How was this patch tested? Added unit tests. Author: Liang-Chi Hsieh Closes #18416 from viirya/SPARK-21204. --- .../spark/sql/catalyst/ScalaReflection.scala | 28 +++++++++++++++-- .../expressions/objects/objects.scala | 5 +-- .../org/apache/spark/sql/SQLImplicits.scala | 10 ++++++ .../spark/sql/DataFrameAggregateSuite.scala | 10 ++++++ .../spark/sql/DatasetPrimitiveSuite.scala | 31 +++++++++++++++++++ 5 files changed, 79 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 814f2c10b909..4d5401f30d39 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 @@ -309,7 +309,10 @@ object ScalaReflection extends ScalaReflection { Invoke(arrayData, primitiveMethod, arrayCls, returnNullable = false) } - case t if t <:< localTypeOf[Seq[_]] => + // We serialize a `Set` to Catalyst array. When we deserialize a Catalyst array + // to a `Set`, if there are duplicated elements, the elements will be de-duplicated. + case t if t <:< localTypeOf[Seq[_]] || + t <:< localTypeOf[scala.collection.Set[_]] => val TypeRef(_, _, Seq(elementType)) = t val Schema(dataType, elementNullable) = schemaFor(elementType) val className = getClassNameFromType(elementType) @@ -327,8 +330,10 @@ object ScalaReflection extends ScalaReflection { } val companion = t.normalize.typeSymbol.companionSymbol.typeSignature - val cls = companion.declaration(newTermName("newBuilder")) match { - case NoSymbol => classOf[Seq[_]] + val cls = companion.member(newTermName("newBuilder")) match { + case NoSymbol if t <:< localTypeOf[Seq[_]] => classOf[Seq[_]] + case NoSymbol if t <:< localTypeOf[scala.collection.Set[_]] => + classOf[scala.collection.Set[_]] case _ => mirror.runtimeClass(t.typeSymbol.asClass) } UnresolvedMapObjects(mapFunction, getPath, Some(cls)) @@ -502,6 +507,19 @@ object ScalaReflection extends ScalaReflection { serializerFor(_, valueType, valuePath, seenTypeSet), valueNullable = !valueType.typeSymbol.asClass.isPrimitive) + case t if t <:< localTypeOf[scala.collection.Set[_]] => + val TypeRef(_, _, Seq(elementType)) = t + + // There's no corresponding Catalyst type for `Set`, we serialize a `Set` to Catalyst array. + // Note that the property of `Set` is only kept when manipulating the data as domain object. + val newInput = + Invoke( + inputObject, + "toSeq", + ObjectType(classOf[Seq[_]])) + + toCatalystArray(newInput, elementType) + case t if t <:< localTypeOf[String] => StaticInvoke( classOf[UTF8String], @@ -713,6 +731,10 @@ object ScalaReflection extends ScalaReflection { val Schema(valueDataType, valueNullable) = schemaFor(valueType) Schema(MapType(schemaFor(keyType).dataType, valueDataType, valueContainsNull = valueNullable), nullable = true) + case t if t <:< localTypeOf[Set[_]] => + val TypeRef(_, _, Seq(elementType)) = t + val Schema(dataType, nullable) = schemaFor(elementType) + Schema(ArrayType(dataType, containsNull = nullable), nullable = true) case t if t <:< localTypeOf[String] => Schema(StringType, nullable = true) case t if t <:< localTypeOf[java.sql.Timestamp] => Schema(TimestampType, nullable = true) case t if t <:< localTypeOf[java.sql.Date] => Schema(DateType, nullable = true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 24c06d8b14b5..9b28a18035b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -627,8 +627,9 @@ case class MapObjects private( val (initCollection, addElement, getResult): (String, String => String, String) = customCollectionCls match { - case Some(cls) if classOf[Seq[_]].isAssignableFrom(cls) => - // Scala sequence + case Some(cls) if classOf[Seq[_]].isAssignableFrom(cls) || + classOf[scala.collection.Set[_]].isAssignableFrom(cls) => + // Scala sequence or set val getBuilder = s"${cls.getName}$$.MODULE$$.newBuilder()" val builder = ctx.freshName("collectionBuilder") ( 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 86574e2f71d9..05db292bd41b 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 @@ -171,6 +171,16 @@ abstract class SQLImplicits extends LowPrioritySQLImplicits { /** @since 2.3.0 */ implicit def newMapEncoder[T <: Map[_, _] : TypeTag]: Encoder[T] = ExpressionEncoder() + /** + * Notice that we serialize `Set` to Catalyst array. The set property is only kept when + * manipulating the domain objects. The serialization format doesn't keep the set property. + * When we have a Catalyst array which contains duplicated elements and convert it to + * `Dataset[Set[T]]` by using the encoder, the elements will be de-duplicated. + * + * @since 2.3.0 + */ + implicit def newSetEncoder[T <: Set[_] : TypeTag]: Encoder[T] = ExpressionEncoder() + // Arrays /** @since 1.6.1 */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 5db354d79bb6..b52d50b195bc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -460,6 +460,16 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { df.select(collect_set($"a"), collect_set($"b")), Seq(Row(Seq(1, 2, 3), Seq(2, 4))) ) + + checkDataset( + df.select(collect_set($"a").as("aSet")).as[Set[Int]], + Set(1, 2, 3)) + checkDataset( + df.select(collect_set($"b").as("bSet")).as[Set[Int]], + Set(2, 4)) + checkDataset( + df.select(collect_set($"a"), collect_set($"b")).as[(Set[Int], Set[Int])], + Seq(Set(1, 2, 3) -> Set(2, 4)): _*) } test("collect functions structs") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala index a6847dcfbffc..f62f9e23db66 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import scala.collection.immutable.{HashSet => HSet} import scala.collection.immutable.Queue import scala.collection.mutable.{LinkedHashMap => LHMap} import scala.collection.mutable.ArrayBuffer @@ -342,6 +343,31 @@ class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext { LHMapClass(LHMap(1 -> 2)) -> LHMap("test" -> MapClass(Map(3 -> 4)))) } + test("arbitrary sets") { + checkDataset(Seq(Set(1, 2, 3, 4)).toDS(), Set(1, 2, 3, 4)) + checkDataset(Seq(Set(1.toLong, 2.toLong)).toDS(), Set(1.toLong, 2.toLong)) + checkDataset(Seq(Set(1.toDouble, 2.toDouble)).toDS(), Set(1.toDouble, 2.toDouble)) + checkDataset(Seq(Set(1.toFloat, 2.toFloat)).toDS(), Set(1.toFloat, 2.toFloat)) + checkDataset(Seq(Set(1.toByte, 2.toByte)).toDS(), Set(1.toByte, 2.toByte)) + checkDataset(Seq(Set(1.toShort, 2.toShort)).toDS(), Set(1.toShort, 2.toShort)) + checkDataset(Seq(Set(true, false)).toDS(), Set(true, false)) + checkDataset(Seq(Set("test1", "test2")).toDS(), Set("test1", "test2")) + checkDataset(Seq(Set(Tuple1(1), Tuple1(2))).toDS(), Set(Tuple1(1), Tuple1(2))) + + checkDataset(Seq(HSet(1, 2)).toDS(), HSet(1, 2)) + checkDataset(Seq(HSet(1.toLong, 2.toLong)).toDS(), HSet(1.toLong, 2.toLong)) + checkDataset(Seq(HSet(1.toDouble, 2.toDouble)).toDS(), HSet(1.toDouble, 2.toDouble)) + checkDataset(Seq(HSet(1.toFloat, 2.toFloat)).toDS(), HSet(1.toFloat, 2.toFloat)) + checkDataset(Seq(HSet(1.toByte, 2.toByte)).toDS(), HSet(1.toByte, 2.toByte)) + checkDataset(Seq(HSet(1.toShort, 2.toShort)).toDS(), HSet(1.toShort, 2.toShort)) + checkDataset(Seq(HSet(true, false)).toDS(), HSet(true, false)) + checkDataset(Seq(HSet("test1", "test2")).toDS(), HSet("test1", "test2")) + checkDataset(Seq(HSet(Tuple1(1), Tuple1(2))).toDS(), HSet(Tuple1(1), Tuple1(2))) + + checkDataset(Seq(Seq(Some(1), None), Seq(Some(2))).toDF("c").as[Set[Integer]], + Seq(Set[Integer](1, null), Set[Integer](2)): _*) + } + test("nested sequences") { checkDataset(Seq(Seq(Seq(1))).toDS(), Seq(Seq(1))) checkDataset(Seq(List(Queue(1))).toDS(), List(Queue(1))) @@ -352,6 +378,11 @@ class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext { checkDataset(Seq(LHMap(Map(1 -> 2) -> 3)).toDS(), LHMap(Map(1 -> 2) -> 3)) } + test("nested set") { + checkDataset(Seq(Set(HSet(1, 2), HSet(3, 4))).toDS(), Set(HSet(1, 2), HSet(3, 4))) + checkDataset(Seq(HSet(Set(1, 2), Set(3, 4))).toDS(), HSet(Set(1, 2), Set(3, 4))) + } + test("package objects") { import packageobject._ checkDataset(Seq(PackageClass(1)).toDS(), PackageClass(1)) From bf66335acab3c0c188f6c378eb8aa6948a259cb2 Mon Sep 17 00:00:00 2001 From: Wang Gengliang Date: Thu, 6 Jul 2017 13:58:27 -0700 Subject: [PATCH 058/125] [SPARK-21323][SQL] Rename plans.logical.statsEstimation.Range to ValueInterval ## What changes were proposed in this pull request? Rename org.apache.spark.sql.catalyst.plans.logical.statsEstimation.Range to ValueInterval. The current naming is identical to logical operator "range". Refactoring it to ValueInterval is more accurate. ## How was this patch tested? unit test Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Wang Gengliang Closes #18549 from gengliangwang/ValueInterval. --- .../statsEstimation/FilterEstimation.scala | 36 ++++++++-------- .../statsEstimation/JoinEstimation.scala | 14 +++---- .../{Range.scala => ValueInterval.scala} | 41 ++++++++++--------- 3 files changed, 48 insertions(+), 43 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/{Range.scala => ValueInterval.scala} (65%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala index 5a3bee7b9e44..e13db85c7a76 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala @@ -316,8 +316,8 @@ case class FilterEstimation(plan: Filter) extends Logging { // decide if the value is in [min, max] of the column. // We currently don't store min/max for binary/string type. // Hence, we assume it is in boundary for binary/string type. - val statsRange = Range(colStat.min, colStat.max, attr.dataType) - if (statsRange.contains(literal)) { + val statsInterval = ValueInterval(colStat.min, colStat.max, attr.dataType) + if (statsInterval.contains(literal)) { if (update) { // We update ColumnStat structure after apply this equality predicate: // Set distinctCount to 1, nullCount to 0, and min/max values (if exist) to the literal @@ -388,9 +388,10 @@ case class FilterEstimation(plan: Filter) extends Logging { // use [min, max] to filter the original hSet dataType match { case _: NumericType | BooleanType | DateType | TimestampType => - val statsRange = Range(colStat.min, colStat.max, dataType).asInstanceOf[NumericRange] + val statsInterval = + ValueInterval(colStat.min, colStat.max, dataType).asInstanceOf[NumericValueInterval] val validQuerySet = hSet.filter { v => - v != null && statsRange.contains(Literal(v, dataType)) + v != null && statsInterval.contains(Literal(v, dataType)) } if (validQuerySet.isEmpty) { @@ -440,12 +441,13 @@ case class FilterEstimation(plan: Filter) extends Logging { update: Boolean): Option[BigDecimal] = { val colStat = colStatsMap(attr) - val statsRange = Range(colStat.min, colStat.max, attr.dataType).asInstanceOf[NumericRange] - val max = statsRange.max.toBigDecimal - val min = statsRange.min.toBigDecimal + val statsInterval = + ValueInterval(colStat.min, colStat.max, attr.dataType).asInstanceOf[NumericValueInterval] + val max = statsInterval.max.toBigDecimal + val min = statsInterval.min.toBigDecimal val ndv = BigDecimal(colStat.distinctCount) - // determine the overlapping degree between predicate range and column's range + // determine the overlapping degree between predicate interval and column's interval val numericLiteral = if (literal.dataType == BooleanType) { if (literal.value.asInstanceOf[Boolean]) BigDecimal(1) else BigDecimal(0) } else { @@ -566,18 +568,18 @@ case class FilterEstimation(plan: Filter) extends Logging { } val colStatLeft = colStatsMap(attrLeft) - val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType) - .asInstanceOf[NumericRange] - val maxLeft = statsRangeLeft.max - val minLeft = statsRangeLeft.min + val statsIntervalLeft = ValueInterval(colStatLeft.min, colStatLeft.max, attrLeft.dataType) + .asInstanceOf[NumericValueInterval] + val maxLeft = statsIntervalLeft.max + val minLeft = statsIntervalLeft.min val colStatRight = colStatsMap(attrRight) - val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType) - .asInstanceOf[NumericRange] - val maxRight = statsRangeRight.max - val minRight = statsRangeRight.min + val statsIntervalRight = ValueInterval(colStatRight.min, colStatRight.max, attrRight.dataType) + .asInstanceOf[NumericValueInterval] + val maxRight = statsIntervalRight.max + val minRight = statsIntervalRight.min - // determine the overlapping degree between predicate range and column's range + // determine the overlapping degree between predicate interval and column's interval val allNotNull = (colStatLeft.nullCount == 0) && (colStatRight.nullCount == 0) val (noOverlap: Boolean, completeOverlap: Boolean) = op match { // Left < Right or Left <= Right diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala index f48196997a24..dcbe36da91df 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala @@ -175,9 +175,9 @@ case class InnerOuterEstimation(join: Join) extends Logging { // Check if the two sides are disjoint val leftKeyStats = leftStats.attributeStats(leftKey) val rightKeyStats = rightStats.attributeStats(rightKey) - val lRange = Range(leftKeyStats.min, leftKeyStats.max, leftKey.dataType) - val rRange = Range(rightKeyStats.min, rightKeyStats.max, rightKey.dataType) - if (Range.isIntersected(lRange, rRange)) { + val lInterval = ValueInterval(leftKeyStats.min, leftKeyStats.max, leftKey.dataType) + val rInterval = ValueInterval(rightKeyStats.min, rightKeyStats.max, rightKey.dataType) + if (ValueInterval.isIntersected(lInterval, rInterval)) { // Get the largest ndv among pairs of join keys val maxNdv = leftKeyStats.distinctCount.max(rightKeyStats.distinctCount) if (maxNdv > ndvDenom) ndvDenom = maxNdv @@ -239,16 +239,16 @@ case class InnerOuterEstimation(join: Join) extends Logging { joinKeyPairs.foreach { case (leftKey, rightKey) => val leftKeyStats = leftStats.attributeStats(leftKey) val rightKeyStats = rightStats.attributeStats(rightKey) - val lRange = Range(leftKeyStats.min, leftKeyStats.max, leftKey.dataType) - val rRange = Range(rightKeyStats.min, rightKeyStats.max, rightKey.dataType) + val lInterval = ValueInterval(leftKeyStats.min, leftKeyStats.max, leftKey.dataType) + val rInterval = ValueInterval(rightKeyStats.min, rightKeyStats.max, rightKey.dataType) // When we reach here, join selectivity is not zero, so each pair of join keys should be // intersected. - assert(Range.isIntersected(lRange, rRange)) + assert(ValueInterval.isIntersected(lInterval, rInterval)) // Update intersected column stats assert(leftKey.dataType.sameType(rightKey.dataType)) val newNdv = leftKeyStats.distinctCount.min(rightKeyStats.distinctCount) - val (newMin, newMax) = Range.intersect(lRange, rRange, leftKey.dataType) + val (newMin, newMax) = ValueInterval.intersect(lInterval, rInterval, leftKey.dataType) val newMaxLen = math.min(leftKeyStats.maxLen, rightKeyStats.maxLen) val newAvgLen = (leftKeyStats.avgLen + rightKeyStats.avgLen) / 2 val newStats = ColumnStat(newNdv, newMin, newMax, 0, newAvgLen, newMaxLen) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/Range.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ValueInterval.scala similarity index 65% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/Range.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ValueInterval.scala index 4ac5ba5689f8..0caaf796a3b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/Range.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ValueInterval.scala @@ -22,12 +22,12 @@ import org.apache.spark.sql.types._ /** Value range of a column. */ -trait Range { +trait ValueInterval { def contains(l: Literal): Boolean } -/** For simplicity we use decimal to unify operations of numeric ranges. */ -case class NumericRange(min: Decimal, max: Decimal) extends Range { +/** For simplicity we use decimal to unify operations of numeric intervals. */ +case class NumericValueInterval(min: Decimal, max: Decimal) extends ValueInterval { override def contains(l: Literal): Boolean = { val lit = EstimationUtils.toDecimal(l.value, l.dataType) min <= lit && max >= lit @@ -38,46 +38,49 @@ case class NumericRange(min: Decimal, max: Decimal) extends Range { * This version of Spark does not have min/max for binary/string types, we define their default * behaviors by this class. */ -class DefaultRange extends Range { +class DefaultValueInterval extends ValueInterval { override def contains(l: Literal): Boolean = true } /** This is for columns with only null values. */ -class NullRange extends Range { +class NullValueInterval extends ValueInterval { override def contains(l: Literal): Boolean = false } -object Range { - def apply(min: Option[Any], max: Option[Any], dataType: DataType): Range = dataType match { - case StringType | BinaryType => new DefaultRange() - case _ if min.isEmpty || max.isEmpty => new NullRange() +object ValueInterval { + def apply( + min: Option[Any], + max: Option[Any], + dataType: DataType): ValueInterval = dataType match { + case StringType | BinaryType => new DefaultValueInterval() + case _ if min.isEmpty || max.isEmpty => new NullValueInterval() case _ => - NumericRange( + NumericValueInterval( min = EstimationUtils.toDecimal(min.get, dataType), max = EstimationUtils.toDecimal(max.get, dataType)) } - def isIntersected(r1: Range, r2: Range): Boolean = (r1, r2) match { - case (_, _: DefaultRange) | (_: DefaultRange, _) => - // The DefaultRange represents string/binary types which do not have max/min stats, + def isIntersected(r1: ValueInterval, r2: ValueInterval): Boolean = (r1, r2) match { + case (_, _: DefaultValueInterval) | (_: DefaultValueInterval, _) => + // The DefaultValueInterval represents string/binary types which do not have max/min stats, // we assume they are intersected to be conservative on estimation true - case (_, _: NullRange) | (_: NullRange, _) => + case (_, _: NullValueInterval) | (_: NullValueInterval, _) => false - case (n1: NumericRange, n2: NumericRange) => + case (n1: NumericValueInterval, n2: NumericValueInterval) => n1.min.compareTo(n2.max) <= 0 && n1.max.compareTo(n2.min) >= 0 } /** - * Intersected results of two ranges. This is only for two overlapped ranges. + * Intersected results of two intervals. This is only for two overlapped intervals. * The outputs are the intersected min/max values. */ - def intersect(r1: Range, r2: Range, dt: DataType): (Option[Any], Option[Any]) = { + def intersect(r1: ValueInterval, r2: ValueInterval, dt: DataType): (Option[Any], Option[Any]) = { (r1, r2) match { - case (_, _: DefaultRange) | (_: DefaultRange, _) => + case (_, _: DefaultValueInterval) | (_: DefaultValueInterval, _) => // binary/string types don't support intersecting. (None, None) - case (n1: NumericRange, n2: NumericRange) => + case (n1: NumericValueInterval, n2: NumericValueInterval) => // Choose the maximum of two min values, and the minimum of two max values. val newMin = if (n1.min <= n2.min) n2.min else n1.min val newMax = if (n1.max <= n2.max) n1.max else n2.max From 0217dfd26f89133f146197359b556c9bf5aca172 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 6 Jul 2017 17:28:20 -0700 Subject: [PATCH 059/125] [SPARK-21267][SS][DOCS] Update Structured Streaming Documentation ## What changes were proposed in this pull request? Few changes to the Structured Streaming documentation - Clarify that the entire stream input table is not materialized - Add information for Ganglia - Add Kafka Sink to the main docs - Removed a couple of leftover experimental tags - Added more associated reading material and talk videos. In addition, https://github.com/apache/spark/pull/16856 broke the link to the RDD programming guide in several places while renaming the page. This PR fixes those sameeragarwal cloud-fan. - Added a redirection to avoid breaking internal and possible external links. - Removed unnecessary redirection pages that were there since the separate scala, java, and python programming guides were merged together in 2013 or 2014. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Tathagata Das Closes #18485 from tdas/SPARK-21267. --- docs/_layouts/global.html | 7 +- docs/index.md | 13 +- docs/java-programming-guide.md | 7 - docs/programming-guide.md | 7 + docs/python-programming-guide.md | 7 - docs/rdd-programming-guide.md | 2 +- docs/scala-programming-guide.md | 7 - docs/sql-programming-guide.md | 16 +- .../structured-streaming-programming-guide.md | 172 +++++++++++++++--- .../scala/org/apache/spark/sql/Dataset.scala | 3 - 10 files changed, 169 insertions(+), 72 deletions(-) delete mode 100644 docs/java-programming-guide.md create mode 100644 docs/programming-guide.md delete mode 100644 docs/python-programming-guide.md delete mode 100644 docs/scala-programming-guide.md diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index c00d0db63cd1..570483c0b04e 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -69,11 +69,10 @@ Programming Guides