From d4007776f2dd85f03f3811ab8ca711f221f62c00 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 25 May 2020 09:18:08 -0500 Subject: [PATCH 01/14] [SPARK-31734][ML][PYSPARK] Add weight support in ClusteringEvaluator ### What changes were proposed in this pull request? Add weight support in ClusteringEvaluator ### Why are the changes needed? Currently, BinaryClassificationEvaluator, RegressionEvaluator, and MulticlassClassificationEvaluator support instance weight, but ClusteringEvaluator doesn't, so we will add instance weight support in ClusteringEvaluator. ### Does this PR introduce _any_ user-facing change? Yes. ClusteringEvaluator.setWeightCol ### How was this patch tested? add new unit test Closes #28553 from huaxingao/weight_evaluator. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../ml/evaluation/ClusteringEvaluator.scala | 34 ++++- .../ml/evaluation/ClusteringMetrics.scala | 128 ++++++++++-------- .../evaluation/ClusteringEvaluatorSuite.scala | 43 +++++- python/pyspark/ml/evaluation.py | 29 +++- 4 files changed, 167 insertions(+), 67 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala index 63b99a0de4b65..19790fd270619 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala @@ -19,10 +19,11 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.Since import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators} -import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol} +import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol, HasWeightCol} import org.apache.spark.ml.util._ import org.apache.spark.sql.Dataset -import org.apache.spark.sql.functions.col +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.DoubleType /** * Evaluator for clustering results. @@ -34,7 +35,8 @@ import org.apache.spark.sql.functions.col */ @Since("2.3.0") class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val uid: String) - extends Evaluator with HasPredictionCol with HasFeaturesCol with DefaultParamsWritable { + extends Evaluator with HasPredictionCol with HasFeaturesCol with HasWeightCol + with DefaultParamsWritable { @Since("2.3.0") def this() = this(Identifiable.randomUID("cluEval")) @@ -53,6 +55,10 @@ class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val uid: Str @Since("2.3.0") def setFeaturesCol(value: String): this.type = set(featuresCol, value) + /** @group setParam */ + @Since("3.1.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + /** * param for metric name in evaluation * (supports `"silhouette"` (default)) @@ -116,12 +122,26 @@ class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val uid: Str */ @Since("3.1.0") def getMetrics(dataset: Dataset[_]): ClusteringMetrics = { - SchemaUtils.validateVectorCompatibleColumn(dataset.schema, $(featuresCol)) - SchemaUtils.checkNumericType(dataset.schema, $(predictionCol)) + val schema = dataset.schema + SchemaUtils.validateVectorCompatibleColumn(schema, $(featuresCol)) + SchemaUtils.checkNumericType(schema, $(predictionCol)) + if (isDefined(weightCol)) { + SchemaUtils.checkNumericType(schema, $(weightCol)) + } + + val weightColName = if (!isDefined(weightCol)) "weightCol" else $(weightCol) val vectorCol = DatasetUtils.columnToVector(dataset, $(featuresCol)) - val df = dataset.select(col($(predictionCol)), - vectorCol.as($(featuresCol), dataset.schema($(featuresCol)).metadata)) + val df = if (!isDefined(weightCol) || $(weightCol).isEmpty) { + dataset.select(col($(predictionCol)), + vectorCol.as($(featuresCol), dataset.schema($(featuresCol)).metadata), + lit(1.0).as(weightColName)) + } else { + dataset.select(col($(predictionCol)), + vectorCol.as($(featuresCol), dataset.schema($(featuresCol)).metadata), + col(weightColName).cast(DoubleType)) + } + val metrics = new ClusteringMetrics(df) metrics.setDistanceMeasure($(distanceMeasure)) metrics diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringMetrics.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringMetrics.scala index 30970337d7d3b..8bf4ee1ecadfb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringMetrics.scala @@ -47,9 +47,9 @@ class ClusteringMetrics private[spark](dataset: Dataset[_]) { val columns = dataset.columns.toSeq if (distanceMeasure.equalsIgnoreCase("squaredEuclidean")) { SquaredEuclideanSilhouette.computeSilhouetteScore( - dataset, columns(0), columns(1)) + dataset, columns(0), columns(1), columns(2)) } else { - CosineSilhouette.computeSilhouetteScore(dataset, columns(0), columns(1)) + CosineSilhouette.computeSilhouetteScore(dataset, columns(0), columns(1), columns(2)) } } } @@ -63,9 +63,10 @@ private[evaluation] abstract class Silhouette { def pointSilhouetteCoefficient( clusterIds: Set[Double], pointClusterId: Double, - pointClusterNumOfPoints: Long, + weightSum: Double, + weight: Double, averageDistanceToCluster: (Double) => Double): Double = { - if (pointClusterNumOfPoints == 1) { + if (weightSum == weight) { // Single-element clusters have silhouette 0 0.0 } else { @@ -77,8 +78,8 @@ private[evaluation] abstract class Silhouette { val neighboringClusterDissimilarity = otherClusterIds.map(averageDistanceToCluster).min // adjustment for excluding the node itself from the computation of the average dissimilarity val currentClusterDissimilarity = - averageDistanceToCluster(pointClusterId) * pointClusterNumOfPoints / - (pointClusterNumOfPoints - 1) + averageDistanceToCluster(pointClusterId) * weightSum / + (weightSum - weight) if (currentClusterDissimilarity < neighboringClusterDissimilarity) { 1 - (currentClusterDissimilarity / neighboringClusterDissimilarity) } else if (currentClusterDissimilarity > neighboringClusterDissimilarity) { @@ -92,8 +93,8 @@ private[evaluation] abstract class Silhouette { /** * Compute the mean Silhouette values of all samples. */ - def overallScore(df: DataFrame, scoreColumn: Column): Double = { - df.select(avg(scoreColumn)).collect()(0).getDouble(0) + def overallScore(df: DataFrame, scoreColumn: Column, weightColumn: Column): Double = { + df.select(sum(scoreColumn * weightColumn) / sum(weightColumn)).collect()(0).getDouble(0) } } @@ -267,7 +268,7 @@ private[evaluation] object SquaredEuclideanSilhouette extends Silhouette { } } - case class ClusterStats(featureSum: Vector, squaredNormSum: Double, numOfPoints: Long) + case class ClusterStats(featureSum: Vector, squaredNormSum: Double, weightSum: Double) /** * The method takes the input dataset and computes the aggregated values @@ -277,6 +278,7 @@ private[evaluation] object SquaredEuclideanSilhouette extends Silhouette { * @param predictionCol The name of the column which contains the predicted cluster id * for the point. * @param featuresCol The name of the column which contains the feature vector of the point. + * @param weightCol The name of the column which contains the instance weight. * @return A [[scala.collection.immutable.Map]] which associates each cluster id * to a [[ClusterStats]] object (which contains the precomputed values `N`, * `$\Psi_{\Gamma}$` and `$Y_{\Gamma}$` for a cluster). @@ -284,36 +286,39 @@ private[evaluation] object SquaredEuclideanSilhouette extends Silhouette { def computeClusterStats( df: DataFrame, predictionCol: String, - featuresCol: String): Map[Double, ClusterStats] = { + featuresCol: String, + weightCol: String): Map[Double, ClusterStats] = { val numFeatures = MetadataUtils.getNumFeatures(df, featuresCol) val clustersStatsRDD = df.select( - col(predictionCol).cast(DoubleType), col(featuresCol), col("squaredNorm")) + col(predictionCol).cast(DoubleType), col(featuresCol), col("squaredNorm"), col(weightCol)) .rdd - .map { row => (row.getDouble(0), (row.getAs[Vector](1), row.getDouble(2))) } - .aggregateByKey[(DenseVector, Double, Long)]((Vectors.zeros(numFeatures).toDense, 0.0, 0L))( - seqOp = { - case ( - (featureSum: DenseVector, squaredNormSum: Double, numOfPoints: Long), - (features, squaredNorm) - ) => - BLAS.axpy(1.0, features, featureSum) - (featureSum, squaredNormSum + squaredNorm, numOfPoints + 1) - }, - combOp = { - case ( - (featureSum1, squaredNormSum1, numOfPoints1), - (featureSum2, squaredNormSum2, numOfPoints2) - ) => - BLAS.axpy(1.0, featureSum2, featureSum1) - (featureSum1, squaredNormSum1 + squaredNormSum2, numOfPoints1 + numOfPoints2) - } - ) + .map { row => (row.getDouble(0), (row.getAs[Vector](1), row.getDouble(2), row.getDouble(3))) } + .aggregateByKey + [(DenseVector, Double, Double)]((Vectors.zeros(numFeatures).toDense, 0.0, 0.0))( + seqOp = { + case ( + (featureSum: DenseVector, squaredNormSum: Double, weightSum: Double), + (features, squaredNorm, weight) + ) => + require(weight >= 0.0, s"illegal weight value: $weight. weight must be >= 0.0.") + BLAS.axpy(weight, features, featureSum) + (featureSum, squaredNormSum + squaredNorm * weight, weightSum + weight) + }, + combOp = { + case ( + (featureSum1, squaredNormSum1, weightSum1), + (featureSum2, squaredNormSum2, weightSum2) + ) => + BLAS.axpy(1.0, featureSum2, featureSum1) + (featureSum1, squaredNormSum1 + squaredNormSum2, weightSum1 + weightSum2) + } + ) clustersStatsRDD .collectAsMap() .mapValues { - case (featureSum: DenseVector, squaredNormSum: Double, numOfPoints: Long) => - SquaredEuclideanSilhouette.ClusterStats(featureSum, squaredNormSum, numOfPoints) + case (featureSum: DenseVector, squaredNormSum: Double, weightSum: Double) => + SquaredEuclideanSilhouette.ClusterStats(featureSum, squaredNormSum, weightSum) } .toMap } @@ -324,6 +329,7 @@ private[evaluation] object SquaredEuclideanSilhouette extends Silhouette { * @param broadcastedClustersMap A map of the precomputed values for each cluster. * @param point The [[org.apache.spark.ml.linalg.Vector]] representing the current point. * @param clusterId The id of the cluster the current point belongs to. + * @param weight The instance weight of the current point. * @param squaredNorm The `$\Xi_{X}$` (which is the squared norm) precomputed for the point. * @return The Silhouette for the point. */ @@ -331,6 +337,7 @@ private[evaluation] object SquaredEuclideanSilhouette extends Silhouette { broadcastedClustersMap: Broadcast[Map[Double, ClusterStats]], point: Vector, clusterId: Double, + weight: Double, squaredNorm: Double): Double = { def compute(targetClusterId: Double): Double = { @@ -338,13 +345,14 @@ private[evaluation] object SquaredEuclideanSilhouette extends Silhouette { val pointDotClusterFeaturesSum = BLAS.dot(point, clusterStats.featureSum) squaredNorm + - clusterStats.squaredNormSum / clusterStats.numOfPoints - - 2 * pointDotClusterFeaturesSum / clusterStats.numOfPoints + clusterStats.squaredNormSum / clusterStats.weightSum - + 2 * pointDotClusterFeaturesSum / clusterStats.weightSum } pointSilhouetteCoefficient(broadcastedClustersMap.value.keySet, clusterId, - broadcastedClustersMap.value(clusterId).numOfPoints, + broadcastedClustersMap.value(clusterId).weightSum, + weight, compute) } @@ -355,12 +363,14 @@ private[evaluation] object SquaredEuclideanSilhouette extends Silhouette { * @param predictionCol The name of the column which contains the predicted cluster id * for the point. * @param featuresCol The name of the column which contains the feature vector of the point. + * @param weightCol The name of the column which contains instance weight. * @return The average of the Silhouette values of the clustered data. */ def computeSilhouetteScore( dataset: Dataset[_], predictionCol: String, - featuresCol: String): Double = { + featuresCol: String, + weightCol: String): Double = { SquaredEuclideanSilhouette.registerKryoClasses(dataset.sparkSession.sparkContext) val squaredNormUDF = udf { @@ -370,7 +380,7 @@ private[evaluation] object SquaredEuclideanSilhouette extends Silhouette { // compute aggregate values for clusters needed by the algorithm val clustersStatsMap = SquaredEuclideanSilhouette - .computeClusterStats(dfWithSquaredNorm, predictionCol, featuresCol) + .computeClusterStats(dfWithSquaredNorm, predictionCol, featuresCol, weightCol) // Silhouette is reasonable only when the number of clusters is greater then 1 assert(clustersStatsMap.size > 1, "Number of clusters must be greater than one.") @@ -378,12 +388,12 @@ private[evaluation] object SquaredEuclideanSilhouette extends Silhouette { val bClustersStatsMap = dataset.sparkSession.sparkContext.broadcast(clustersStatsMap) val computeSilhouetteCoefficientUDF = udf { - computeSilhouetteCoefficient(bClustersStatsMap, _: Vector, _: Double, _: Double) + computeSilhouetteCoefficient(bClustersStatsMap, _: Vector, _: Double, _: Double, _: Double) } val silhouetteScore = overallScore(dfWithSquaredNorm, computeSilhouetteCoefficientUDF(col(featuresCol), col(predictionCol).cast(DoubleType), - col("squaredNorm"))) + col(weightCol), col("squaredNorm")), col(weightCol)) bClustersStatsMap.destroy() @@ -472,30 +482,35 @@ private[evaluation] object CosineSilhouette extends Silhouette { * about a cluster which are needed by the algorithm. * * @param df The DataFrame which contains the input data + * @param featuresCol The name of the column which contains the feature vector of the point. * @param predictionCol The name of the column which contains the predicted cluster id * for the point. + * @param weightCol The name of the column which contains the instance weight. * @return A [[scala.collection.immutable.Map]] which associates each cluster id to a * its statistics (ie. the precomputed values `N` and `$\Omega_{\Gamma}$`). */ def computeClusterStats( df: DataFrame, featuresCol: String, - predictionCol: String): Map[Double, (Vector, Long)] = { + predictionCol: String, + weightCol: String): Map[Double, (Vector, Double)] = { val numFeatures = MetadataUtils.getNumFeatures(df, featuresCol) val clustersStatsRDD = df.select( - col(predictionCol).cast(DoubleType), col(normalizedFeaturesColName)) + col(predictionCol).cast(DoubleType), col(normalizedFeaturesColName), col(weightCol)) .rdd - .map { row => (row.getDouble(0), row.getAs[Vector](1)) } - .aggregateByKey[(DenseVector, Long)]((Vectors.zeros(numFeatures).toDense, 0L))( + .map { row => (row.getDouble(0), (row.getAs[Vector](1), row.getDouble(2))) } + .aggregateByKey[(DenseVector, Double)]((Vectors.zeros(numFeatures).toDense, 0.0))( seqOp = { - case ((normalizedFeaturesSum: DenseVector, numOfPoints: Long), (normalizedFeatures)) => - BLAS.axpy(1.0, normalizedFeatures, normalizedFeaturesSum) - (normalizedFeaturesSum, numOfPoints + 1) + case ((normalizedFeaturesSum: DenseVector, weightSum: Double), + (normalizedFeatures, weight)) => + require(weight >= 0.0, s"illegal weight value: $weight. weight must be >= 0.0.") + BLAS.axpy(weight, normalizedFeatures, normalizedFeaturesSum) + (normalizedFeaturesSum, weightSum + weight) }, combOp = { - case ((normalizedFeaturesSum1, numOfPoints1), (normalizedFeaturesSum2, numOfPoints2)) => + case ((normalizedFeaturesSum1, weightSum1), (normalizedFeaturesSum2, weightSum2)) => BLAS.axpy(1.0, normalizedFeaturesSum2, normalizedFeaturesSum1) - (normalizedFeaturesSum1, numOfPoints1 + numOfPoints2) + (normalizedFeaturesSum1, weightSum1 + weightSum2) } ) @@ -511,11 +526,13 @@ private[evaluation] object CosineSilhouette extends Silhouette { * @param normalizedFeatures The [[org.apache.spark.ml.linalg.Vector]] representing the * normalized features of the current point. * @param clusterId The id of the cluster the current point belongs to. + * @param weight The instance weight of the current point. */ def computeSilhouetteCoefficient( - broadcastedClustersMap: Broadcast[Map[Double, (Vector, Long)]], + broadcastedClustersMap: Broadcast[Map[Double, (Vector, Double)]], normalizedFeatures: Vector, - clusterId: Double): Double = { + clusterId: Double, + weight: Double): Double = { def compute(targetClusterId: Double): Double = { val (normalizedFeatureSum, numOfPoints) = broadcastedClustersMap.value(targetClusterId) @@ -525,6 +542,7 @@ private[evaluation] object CosineSilhouette extends Silhouette { pointSilhouetteCoefficient(broadcastedClustersMap.value.keySet, clusterId, broadcastedClustersMap.value(clusterId)._2, + weight, compute) } @@ -535,12 +553,14 @@ private[evaluation] object CosineSilhouette extends Silhouette { * @param predictionCol The name of the column which contains the predicted cluster id * for the point. * @param featuresCol The name of the column which contains the feature vector of the point. + * @param weightCol The name of the column which contains the instance weight. * @return The average of the Silhouette values of the clustered data. */ def computeSilhouetteScore( dataset: Dataset[_], predictionCol: String, - featuresCol: String): Double = { + featuresCol: String, + weightCol: String): Double = { val normalizeFeatureUDF = udf { features: Vector => { val norm = Vectors.norm(features, 2.0) @@ -553,7 +573,7 @@ private[evaluation] object CosineSilhouette extends Silhouette { // compute aggregate values for clusters needed by the algorithm val clustersStatsMap = computeClusterStats(dfWithNormalizedFeatures, featuresCol, - predictionCol) + predictionCol, weightCol) // Silhouette is reasonable only when the number of clusters is greater then 1 assert(clustersStatsMap.size > 1, "Number of clusters must be greater than one.") @@ -561,12 +581,12 @@ private[evaluation] object CosineSilhouette extends Silhouette { val bClustersStatsMap = dataset.sparkSession.sparkContext.broadcast(clustersStatsMap) val computeSilhouetteCoefficientUDF = udf { - computeSilhouetteCoefficient(bClustersStatsMap, _: Vector, _: Double) + computeSilhouetteCoefficient(bClustersStatsMap, _: Vector, _: Double, _: Double) } val silhouetteScore = overallScore(dfWithNormalizedFeatures, computeSilhouetteCoefficientUDF(col(normalizedFeaturesColName), - col(predictionCol).cast(DoubleType))) + col(predictionCol).cast(DoubleType), col(weightCol)), col(weightCol)) bClustersStatsMap.destroy() diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala index 29fed5322c9c9..d4c620adc2e3c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala @@ -19,12 +19,13 @@ package org.apache.spark.ml.evaluation import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.attribute.AttributeGroup -import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.functions.lit class ClusteringEvaluatorSuite @@ -161,4 +162,44 @@ class ClusteringEvaluatorSuite assert(evaluator.evaluate(irisDataset) == silhouetteScoreCosin) } + + test("test weight support") { + Seq("squaredEuclidean", "cosine").foreach { distanceMeasure => + val evaluator1 = new ClusteringEvaluator() + .setFeaturesCol("features") + .setPredictionCol("label") + .setDistanceMeasure(distanceMeasure) + + val evaluator2 = new ClusteringEvaluator() + .setFeaturesCol("features") + .setPredictionCol("label") + .setDistanceMeasure(distanceMeasure) + .setWeightCol("weight") + + Seq(0.25, 1.0, 10.0, 99.99).foreach { w => + var score1 = evaluator1.evaluate(irisDataset) + var score2 = evaluator2.evaluate(irisDataset.withColumn("weight", lit(w))) + assert(score1 ~== score2 relTol 1e-6) + + score1 = evaluator1.evaluate(newIrisDataset) + score2 = evaluator2.evaluate(newIrisDataset.withColumn("weight", lit(w))) + assert(score1 ~== score2 relTol 1e-6) + } + } + } + + test("single-element clusters with weight") { + val singleItemClusters = spark.createDataFrame(spark.sparkContext.parallelize(Array( + (0.0, Vectors.dense(5.1, 3.5, 1.4, 0.2), 6.0), + (1.0, Vectors.dense(7.0, 3.2, 4.7, 1.4), 0.25), + (2.0, Vectors.dense(6.3, 3.3, 6.0, 2.5), 9.99)))).toDF("label", "features", "weight") + Seq("squaredEuclidean", "cosine").foreach { distanceMeasure => + val evaluator = new ClusteringEvaluator() + .setFeaturesCol("features") + .setPredictionCol("label") + .setDistanceMeasure(distanceMeasure) + .setWeightCol("weight") + assert(evaluator.evaluate(singleItemClusters) === 0.0) + } + } } diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index 265f02c1a03ac..a69a57f588571 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -654,7 +654,7 @@ def setParams(self, predictionCol="prediction", labelCol="label", @inherit_doc -class ClusteringEvaluator(JavaEvaluator, HasPredictionCol, HasFeaturesCol, +class ClusteringEvaluator(JavaEvaluator, HasPredictionCol, HasFeaturesCol, HasWeightCol, JavaMLReadable, JavaMLWritable): """ Evaluator for Clustering results, which expects two input @@ -677,6 +677,18 @@ class ClusteringEvaluator(JavaEvaluator, HasPredictionCol, HasFeaturesCol, ClusteringEvaluator... >>> evaluator.evaluate(dataset) 0.9079... + >>> featureAndPredictionsWithWeight = map(lambda x: (Vectors.dense(x[0]), x[1], x[2]), + ... [([0.0, 0.5], 0.0, 2.5), ([0.5, 0.0], 0.0, 2.5), ([10.0, 11.0], 1.0, 2.5), + ... ([10.5, 11.5], 1.0, 2.5), ([1.0, 1.0], 0.0, 2.5), ([8.0, 6.0], 1.0, 2.5)]) + >>> dataset = spark.createDataFrame( + ... featureAndPredictionsWithWeight, ["features", "prediction", "weight"]) + >>> evaluator = ClusteringEvaluator() + >>> evaluator.setPredictionCol("prediction") + ClusteringEvaluator... + >>> evaluator.setWeightCol("weight") + ClusteringEvaluator... + >>> evaluator.evaluate(dataset) + 0.9079... >>> ce_path = temp_path + "/ce" >>> evaluator.save(ce_path) >>> evaluator2 = ClusteringEvaluator.load(ce_path) @@ -694,10 +706,10 @@ class ClusteringEvaluator(JavaEvaluator, HasPredictionCol, HasFeaturesCol, @keyword_only def __init__(self, predictionCol="prediction", featuresCol="features", - metricName="silhouette", distanceMeasure="squaredEuclidean"): + metricName="silhouette", distanceMeasure="squaredEuclidean", weightCol=None): """ __init__(self, predictionCol="prediction", featuresCol="features", \ - metricName="silhouette", distanceMeasure="squaredEuclidean") + metricName="silhouette", distanceMeasure="squaredEuclidean", weightCol=None) """ super(ClusteringEvaluator, self).__init__() self._java_obj = self._new_java_obj( @@ -709,10 +721,10 @@ def __init__(self, predictionCol="prediction", featuresCol="features", @keyword_only @since("2.3.0") def setParams(self, predictionCol="prediction", featuresCol="features", - metricName="silhouette", distanceMeasure="squaredEuclidean"): + metricName="silhouette", distanceMeasure="squaredEuclidean", weightCol=None): """ setParams(self, predictionCol="prediction", featuresCol="features", \ - metricName="silhouette", distanceMeasure="squaredEuclidean") + metricName="silhouette", distanceMeasure="squaredEuclidean", weightCol=None) Sets params for clustering evaluator. """ kwargs = self._input_kwargs @@ -758,6 +770,13 @@ def setPredictionCol(self, value): """ return self._set(predictionCol=value) + @since("3.1.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + @inherit_doc class RankingEvaluator(JavaEvaluator, HasLabelCol, HasPredictionCol, From 0df8dd60733066076967f0525210bbdb5e12415a Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 25 May 2020 14:27:47 +0000 Subject: [PATCH 02/14] [SPARK-30352][SQL] DataSourceV2: Add CURRENT_CATALOG function ### What changes were proposed in this pull request? As we support multiple catalogs with DataSourceV2, we may need the `CURRENT_CATALOG` value expression from the SQL standard. `CURRENT_CATALOG` is a general value specification in the SQL Standard, described as: > The value specified by CURRENT_CATALOG is the character string that represents the current default catalog name. ### Why are the changes needed? improve catalog v2 with ANSI SQL standard. ### Does this PR introduce any user-facing change? yes, add a new function `current_catalog()` to point the current active catalog ### How was this patch tested? add ut Closes #27006 from yaooqinn/SPARK-30352. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../spark/sql/catalyst/expressions/misc.scala | 18 ++++++++++++++++++ .../sql/catalyst/optimizer/Optimizer.scala | 4 ++-- .../catalyst/optimizer/finishAnalysis.scala | 10 ++++++++-- .../sql-functions/sql-expression-schema.md | 3 ++- .../inputs/current_database_catalog.sql | 2 ++ .../results/current_database_catalog.sql.out | 10 ++++++++++ 7 files changed, 43 insertions(+), 5 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/current_database_catalog.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/current_database_catalog.sql.out diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 5e53927885ca4..d4799cace4531 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -491,6 +491,7 @@ object FunctionRegistry { expression[InputFileBlockLength]("input_file_block_length"), expression[MonotonicallyIncreasingID]("monotonically_increasing_id"), expression[CurrentDatabase]("current_database"), + expression[CurrentCatalog]("current_catalog"), expression[CallMethodViaReflection]("reflect"), expression[CallMethodViaReflection]("java_method", true), expression[SparkVersion]("version"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index 8ce3ddd30a69e..617ddcb69eab0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -116,6 +116,24 @@ case class CurrentDatabase() extends LeafExpression with Unevaluable { override def prettyName: String = "current_database" } +/** + * Returns the current catalog. + */ +@ExpressionDescription( + usage = "_FUNC_() - Returns the current catalog.", + examples = """ + Examples: + > SELECT _FUNC_(); + spark_catalog + """, + since = "3.1.0") +case class CurrentCatalog() extends LeafExpression with Unevaluable { + override def dataType: DataType = StringType + override def foldable: Boolean = true + override def nullable: Boolean = false + override def prettyName: String = "current_catalog" +} + // scalastyle:off line.size.limit @ExpressionDescription( usage = """_FUNC_() - Returns an universally unique identifier (UUID) string. The value is returned as a canonical UUID 36-character string.""", 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 e59e3b999aa7f..f1a307b1c2cc1 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 @@ -133,7 +133,7 @@ abstract class Optimizer(catalogManager: CatalogManager) ReplaceExpressions, RewriteNonCorrelatedExists, ComputeCurrentTime, - GetCurrentDatabase(catalogManager), + GetCurrentDatabaseAndCatalog(catalogManager), RewriteDistinctAggregates, ReplaceDeduplicateWithAggregate) :: ////////////////////////////////////////////////////////////////////////////////////////// @@ -223,7 +223,7 @@ abstract class Optimizer(catalogManager: CatalogManager) EliminateView.ruleName :: ReplaceExpressions.ruleName :: ComputeCurrentTime.ruleName :: - GetCurrentDatabase(catalogManager).ruleName :: + GetCurrentDatabaseAndCatalog(catalogManager).ruleName :: RewriteDistinctAggregates.ruleName :: ReplaceDeduplicateWithAggregate.ruleName :: ReplaceIntersectWithSemiJoin.ruleName :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index 80d85827657fd..6c9bb6db06d86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -91,15 +91,21 @@ object ComputeCurrentTime extends Rule[LogicalPlan] { } -/** Replaces the expression of CurrentDatabase with the current database name. */ -case class GetCurrentDatabase(catalogManager: CatalogManager) extends Rule[LogicalPlan] { +/** + * Replaces the expression of CurrentDatabase with the current database name. + * Replaces the expression of CurrentCatalog with the current catalog name. + */ +case class GetCurrentDatabaseAndCatalog(catalogManager: CatalogManager) extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ val currentNamespace = catalogManager.currentNamespace.quoted + val currentCatalog = catalogManager.currentCatalog.name() plan transformAllExpressions { case CurrentDatabase() => Literal.create(currentNamespace, StringType) + case CurrentCatalog() => + Literal.create(currentCatalog, StringType) } } } diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index a4f076396c517..23173c8ba1f11 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -1,6 +1,6 @@ ## Summary - - Number of queries: 336 + - Number of queries: 337 - Number of expressions that missing example: 34 - Expressions missing examples: and,string,tinyint,double,smallint,date,decimal,boolean,float,binary,bigint,int,timestamp,cume_dist,dense_rank,input_file_block_length,input_file_block_start,input_file_name,lag,lead,monotonically_increasing_id,ntile,struct,!,not,or,percent_rank,rank,row_number,spark_partition_id,version,window,positive,count_min_sketch ## Schema of Built-in Functions @@ -82,6 +82,7 @@ | org.apache.spark.sql.catalyst.expressions.CsvToStructs | from_csv | SELECT from_csv('1, 0.8', 'a INT, b DOUBLE') | struct> | | org.apache.spark.sql.catalyst.expressions.Cube | cube | SELECT name, age, count(*) FROM VALUES (2, 'Alice'), (5, 'Bob') people(age, name) GROUP BY cube(name, age) | struct | | org.apache.spark.sql.catalyst.expressions.CumeDist | cume_dist | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.CurrentCatalog | current_catalog | SELECT current_catalog() | struct | | org.apache.spark.sql.catalyst.expressions.CurrentDatabase | current_database | SELECT current_database() | struct | | org.apache.spark.sql.catalyst.expressions.CurrentDate | current_date | SELECT current_date() | struct | | org.apache.spark.sql.catalyst.expressions.CurrentTimestamp | current_timestamp | SELECT current_timestamp() | struct | diff --git a/sql/core/src/test/resources/sql-tests/inputs/current_database_catalog.sql b/sql/core/src/test/resources/sql-tests/inputs/current_database_catalog.sql new file mode 100644 index 0000000000000..4406f1bc2e6e3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/current_database_catalog.sql @@ -0,0 +1,2 @@ +-- get current_datebase and current_catalog +select current_database(), current_catalog(); diff --git a/sql/core/src/test/resources/sql-tests/results/current_database_catalog.sql.out b/sql/core/src/test/resources/sql-tests/results/current_database_catalog.sql.out new file mode 100644 index 0000000000000..b714463a0aa0c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/current_database_catalog.sql.out @@ -0,0 +1,10 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 1 + + +-- !query +select current_database(), current_catalog() +-- !query schema +struct +-- !query output +default spark_catalog From 92685c014816e82c339f047f51c5363e32bc7338 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 25 May 2020 15:00:11 +0000 Subject: [PATCH 03/14] [SPARK-31755][SQL][FOLLOWUP] Update date-time, CSV and JSON benchmark results ### What changes were proposed in this pull request? Re-generate results of: - DateTimeBenchmark - CSVBenchmark - JsonBenchmark in the environment: | Item | Description | | ---- | ----| | Region | us-west-2 (Oregon) | | Instance | r3.xlarge | | AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) | | Java | OpenJDK 64-Bit Server VM 1.8.0_242 and OpenJDK 64-Bit Server VM 11.0.6+10 | ### Why are the changes needed? 1. The PR https://github.com/apache/spark/pull/28576 changed date-time parser. The `DateTimeBenchmark` should confirm that the PR didn't slow down date/timestamp parsing. 2. CSV/JSON datasources are affected by the above PR too. This PR updates the benchmark results in the same environment as other benchmarks to have a base line for future optimizations. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running benchmarks via the script: ```python #!/usr/bin/env python3 import os from sparktestsupport.shellutils import run_cmd benchmarks = [ ['sql/test', 'org.apache.spark.sql.execution.benchmark.DateTimeBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.datasources.csv.CSVBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.datasources.json.JsonBenchmark'] ] print('Set SPARK_GENERATE_BENCHMARK_FILES=1') os.environ['SPARK_GENERATE_BENCHMARK_FILES'] = '1' for b in benchmarks: print("Run benchmark: %s" % b[1]) run_cmd(['build/sbt', '%s:runMain %s' % (b[0], b[1])]) ``` Closes #28613 from MaxGekk/missing-hour-year-benchmarks. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../benchmarks/CSVBenchmark-jdk11-results.txt | 88 +++--- sql/core/benchmarks/CSVBenchmark-results.txt | 88 +++--- .../DateTimeBenchmark-jdk11-results.txt | 254 +++++++++--------- .../benchmarks/DateTimeBenchmark-results.txt | 254 +++++++++--------- .../JsonBenchmark-jdk11-results.txt | 130 ++++----- sql/core/benchmarks/JsonBenchmark-results.txt | 130 ++++----- 6 files changed, 472 insertions(+), 472 deletions(-) diff --git a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt index 147a77ff098d0..0e82b632793d2 100644 --- a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt @@ -2,66 +2,66 @@ Benchmark to measure CSV read/write performance ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 24907 29374 NaN 0.0 498130.5 1.0X +One quoted string 46568 46683 198 0.0 931358.6 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 1000 columns 62811 63690 1416 0.0 62811.4 1.0X -Select 100 columns 23839 24064 230 0.0 23839.5 2.6X -Select one column 19936 20641 827 0.1 19936.4 3.2X -count() 4174 4380 206 0.2 4174.4 15.0X -Select 100 columns, one bad input field 41015 42380 1688 0.0 41015.4 1.5X -Select 100 columns, corrupt record field 46281 46338 93 0.0 46280.5 1.4X +Select 1000 columns 129836 130796 1404 0.0 129836.0 1.0X +Select 100 columns 40444 40679 261 0.0 40443.5 3.2X +Select one column 33429 33475 73 0.0 33428.6 3.9X +count() 7967 8047 73 0.1 7966.7 16.3X +Select 100 columns, one bad input field 90639 90832 266 0.0 90638.6 1.4X +Select 100 columns, corrupt record field 109023 109084 74 0.0 109023.3 1.2X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns + count() 10810 10997 163 0.9 1081.0 1.0X -Select 1 column + count() 7608 7641 47 1.3 760.8 1.4X -count() 2415 2462 77 4.1 241.5 4.5X +Select 10 columns + count() 20685 20707 35 0.5 2068.5 1.0X +Select 1 column + count() 13096 13149 49 0.8 1309.6 1.6X +count() 3994 4001 7 2.5 399.4 5.2X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 874 914 37 11.4 87.4 1.0X -to_csv(timestamp) 7051 7223 250 1.4 705.1 0.1X -write timestamps to files 6712 6741 31 1.5 671.2 0.1X -Create a dataset of dates 909 945 35 11.0 90.9 1.0X -to_csv(date) 4222 4231 8 2.4 422.2 0.2X -write dates to files 3799 3813 14 2.6 379.9 0.2X +Create a dataset of timestamps 2169 2203 32 4.6 216.9 1.0X +to_csv(timestamp) 14401 14591 168 0.7 1440.1 0.2X +write timestamps to files 13209 13276 59 0.8 1320.9 0.2X +Create a dataset of dates 2231 2248 17 4.5 223.1 1.0X +to_csv(date) 10406 10473 68 1.0 1040.6 0.2X +write dates to files 7970 7976 9 1.3 797.0 0.3X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 1342 1364 35 7.5 134.2 1.0X -read timestamps from files 20300 20473 247 0.5 2030.0 0.1X -infer timestamps from files 40705 40744 54 0.2 4070.5 0.0X -read date text from files 1146 1151 6 8.7 114.6 1.2X -read date from files 12278 12408 117 0.8 1227.8 0.1X -infer date from files 12734 12872 220 0.8 1273.4 0.1X -timestamp strings 1467 1482 15 6.8 146.7 0.9X -parse timestamps from Dataset[String] 21708 22234 477 0.5 2170.8 0.1X -infer timestamps from Dataset[String] 42357 43253 922 0.2 4235.7 0.0X -date strings 1512 1532 18 6.6 151.2 0.9X -parse dates from Dataset[String] 13436 13470 33 0.7 1343.6 0.1X -from_csv(timestamp) 20390 20486 95 0.5 2039.0 0.1X -from_csv(date) 12592 12693 139 0.8 1259.2 0.1X +read timestamp text from files 2387 2391 6 4.2 238.7 1.0X +read timestamps from files 53503 53593 124 0.2 5350.3 0.0X +infer timestamps from files 107988 108668 647 0.1 10798.8 0.0X +read date text from files 2121 2133 12 4.7 212.1 1.1X +read date from files 29983 30039 48 0.3 2998.3 0.1X +infer date from files 30196 30436 218 0.3 3019.6 0.1X +timestamp strings 3098 3109 10 3.2 309.8 0.8X +parse timestamps from Dataset[String] 63331 63426 84 0.2 6333.1 0.0X +infer timestamps from Dataset[String] 124003 124463 490 0.1 12400.3 0.0X +date strings 3423 3429 11 2.9 342.3 0.7X +parse dates from Dataset[String] 34235 34314 76 0.3 3423.5 0.1X +from_csv(timestamp) 60829 61600 668 0.2 6082.9 0.0X +from_csv(date) 33047 33173 139 0.3 3304.7 0.1X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -w/o filters 12535 12606 67 0.0 125348.8 1.0X -pushdown disabled 12611 12672 91 0.0 126112.9 1.0X -w/ filters 1093 1099 11 0.1 10928.3 11.5X +w/o filters 28752 28765 16 0.0 287516.5 1.0X +pushdown disabled 28856 28880 22 0.0 288556.3 1.0X +w/ filters 1714 1731 15 0.1 17137.3 16.8X diff --git a/sql/core/benchmarks/CSVBenchmark-results.txt b/sql/core/benchmarks/CSVBenchmark-results.txt index 498ca4caa0e45..a3af46c037bf9 100644 --- a/sql/core/benchmarks/CSVBenchmark-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-results.txt @@ -2,66 +2,66 @@ Benchmark to measure CSV read/write performance ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 24073 24109 33 0.0 481463.5 1.0X +One quoted string 45457 45731 344 0.0 909136.8 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 1000 columns 58415 59611 2071 0.0 58414.8 1.0X -Select 100 columns 22568 23020 594 0.0 22568.0 2.6X -Select one column 18995 19058 99 0.1 18995.0 3.1X -count() 5301 5332 30 0.2 5300.9 11.0X -Select 100 columns, one bad input field 39736 40153 361 0.0 39736.1 1.5X -Select 100 columns, corrupt record field 47195 47826 590 0.0 47195.2 1.2X +Select 1000 columns 129646 130527 1412 0.0 129646.3 1.0X +Select 100 columns 42444 42551 119 0.0 42444.0 3.1X +Select one column 35415 35428 20 0.0 35414.6 3.7X +count() 11114 11128 16 0.1 11113.6 11.7X +Select 100 columns, one bad input field 93353 93670 275 0.0 93352.6 1.4X +Select 100 columns, corrupt record field 113569 113952 373 0.0 113568.8 1.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns + count() 9884 9904 25 1.0 988.4 1.0X -Select 1 column + count() 6794 6835 46 1.5 679.4 1.5X -count() 2060 2065 5 4.9 206.0 4.8X +Select 10 columns + count() 18498 18589 87 0.5 1849.8 1.0X +Select 1 column + count() 11078 11095 27 0.9 1107.8 1.7X +count() 3928 3950 22 2.5 392.8 4.7X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 717 732 18 14.0 71.7 1.0X -to_csv(timestamp) 6994 7100 121 1.4 699.4 0.1X -write timestamps to files 6417 6435 27 1.6 641.7 0.1X -Create a dataset of dates 827 855 24 12.1 82.7 0.9X -to_csv(date) 4408 4438 32 2.3 440.8 0.2X -write dates to files 3738 3758 28 2.7 373.8 0.2X +Create a dataset of timestamps 1933 1940 11 5.2 193.3 1.0X +to_csv(timestamp) 18078 18243 255 0.6 1807.8 0.1X +write timestamps to files 12668 12786 134 0.8 1266.8 0.2X +Create a dataset of dates 2196 2201 5 4.6 219.6 0.9X +to_csv(date) 9583 9597 21 1.0 958.3 0.2X +write dates to files 7091 7110 20 1.4 709.1 0.3X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 1121 1176 52 8.9 112.1 1.0X -read timestamps from files 21298 21366 105 0.5 2129.8 0.1X -infer timestamps from files 41008 41051 39 0.2 4100.8 0.0X -read date text from files 962 967 5 10.4 96.2 1.2X -read date from files 11749 11772 22 0.9 1174.9 0.1X -infer date from files 12426 12459 29 0.8 1242.6 0.1X -timestamp strings 1508 1519 9 6.6 150.8 0.7X -parse timestamps from Dataset[String] 21674 21997 455 0.5 2167.4 0.1X -infer timestamps from Dataset[String] 42141 42230 105 0.2 4214.1 0.0X -date strings 1694 1701 8 5.9 169.4 0.7X -parse dates from Dataset[String] 12929 12951 25 0.8 1292.9 0.1X -from_csv(timestamp) 20603 20786 166 0.5 2060.3 0.1X -from_csv(date) 12325 12338 12 0.8 1232.5 0.1X +read timestamp text from files 2166 2177 10 4.6 216.6 1.0X +read timestamps from files 53212 53402 281 0.2 5321.2 0.0X +infer timestamps from files 109788 110372 570 0.1 10978.8 0.0X +read date text from files 1921 1929 8 5.2 192.1 1.1X +read date from files 25470 25499 25 0.4 2547.0 0.1X +infer date from files 27201 27342 134 0.4 2720.1 0.1X +timestamp strings 3638 3653 19 2.7 363.8 0.6X +parse timestamps from Dataset[String] 61894 62532 555 0.2 6189.4 0.0X +infer timestamps from Dataset[String] 125171 125430 236 0.1 12517.1 0.0X +date strings 3736 3749 14 2.7 373.6 0.6X +parse dates from Dataset[String] 30787 30829 43 0.3 3078.7 0.1X +from_csv(timestamp) 60842 61035 209 0.2 6084.2 0.0X +from_csv(date) 30123 30196 95 0.3 3012.3 0.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -w/o filters 12455 12474 22 0.0 124553.8 1.0X -pushdown disabled 12462 12486 29 0.0 124624.9 1.0X -w/ filters 1073 1092 18 0.1 10727.6 11.6X +w/o filters 28985 29042 80 0.0 289852.9 1.0X +pushdown disabled 29080 29146 58 0.0 290799.4 1.0X +w/ filters 2072 2084 17 0.0 20722.3 14.0X diff --git a/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt b/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt index 61ca342a0d559..f4ed8ce4afaea 100644 --- a/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt @@ -6,18 +6,18 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz datetime +/- interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date + interval(m) 1496 1569 104 6.7 149.6 1.0X -date + interval(m, d) 1514 1526 17 6.6 151.4 1.0X -date + interval(m, d, ms) 6231 6253 30 1.6 623.1 0.2X -date - interval(m) 1481 1487 9 6.8 148.1 1.0X -date - interval(m, d) 1550 1552 2 6.5 155.0 1.0X -date - interval(m, d, ms) 6269 6272 4 1.6 626.9 0.2X -timestamp + interval(m) 3017 3056 54 3.3 301.7 0.5X -timestamp + interval(m, d) 3146 3148 3 3.2 314.6 0.5X -timestamp + interval(m, d, ms) 3446 3460 20 2.9 344.6 0.4X -timestamp - interval(m) 3045 3059 19 3.3 304.5 0.5X -timestamp - interval(m, d) 3147 3164 25 3.2 314.7 0.5X -timestamp - interval(m, d, ms) 3425 3442 25 2.9 342.5 0.4X +date + interval(m) 1660 1745 120 6.0 166.0 1.0X +date + interval(m, d) 1672 1685 19 6.0 167.2 1.0X +date + interval(m, d, ms) 6462 6481 27 1.5 646.2 0.3X +date - interval(m) 1456 1480 35 6.9 145.6 1.1X +date - interval(m, d) 1501 1509 11 6.7 150.1 1.1X +date - interval(m, d, ms) 6457 6466 12 1.5 645.7 0.3X +timestamp + interval(m) 2941 2944 4 3.4 294.1 0.6X +timestamp + interval(m, d) 3008 3012 6 3.3 300.8 0.6X +timestamp + interval(m, d, ms) 3329 3333 6 3.0 332.9 0.5X +timestamp - interval(m) 2964 2982 26 3.4 296.4 0.6X +timestamp - interval(m, d) 3030 3039 13 3.3 303.0 0.5X +timestamp - interval(m, d, ms) 3312 3313 1 3.0 331.2 0.5X ================================================================================================ @@ -28,92 +28,92 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp wholestage off 332 336 5 30.1 33.2 1.0X -cast to timestamp wholestage on 333 344 10 30.0 33.3 1.0X +cast to timestamp wholestage off 333 334 0 30.0 33.3 1.0X +cast to timestamp wholestage on 349 368 12 28.6 34.9 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -year of timestamp wholestage off 1246 1257 16 8.0 124.6 1.0X -year of timestamp wholestage on 1209 1218 12 8.3 120.9 1.0X +year of timestamp wholestage off 1229 1229 1 8.1 122.9 1.0X +year of timestamp wholestage on 1218 1223 5 8.2 121.8 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -quarter of timestamp wholestage off 1608 1616 11 6.2 160.8 1.0X -quarter of timestamp wholestage on 1540 1552 10 6.5 154.0 1.0X +quarter of timestamp wholestage off 1593 1594 2 6.3 159.3 1.0X +quarter of timestamp wholestage on 1515 1529 14 6.6 151.5 1.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -month of timestamp wholestage off 1242 1246 6 8.1 124.2 1.0X -month of timestamp wholestage on 1202 1212 11 8.3 120.2 1.0X +month of timestamp wholestage off 1222 1246 34 8.2 122.2 1.0X +month of timestamp wholestage on 1207 1232 31 8.3 120.7 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekofyear of timestamp wholestage off 1879 1885 8 5.3 187.9 1.0X -weekofyear of timestamp wholestage on 1832 1845 10 5.5 183.2 1.0X +weekofyear of timestamp wholestage off 2453 2455 2 4.1 245.3 1.0X +weekofyear of timestamp wholestage on 2357 2380 22 4.2 235.7 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -day of timestamp wholestage off 1236 1239 4 8.1 123.6 1.0X -day of timestamp wholestage on 1206 1219 17 8.3 120.6 1.0X +day of timestamp wholestage off 1216 1219 5 8.2 121.6 1.0X +day of timestamp wholestage on 1205 1221 25 8.3 120.5 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofyear of timestamp wholestage off 1308 1309 1 7.6 130.8 1.0X -dayofyear of timestamp wholestage on 1239 1255 15 8.1 123.9 1.1X +dayofyear of timestamp wholestage off 1268 1274 9 7.9 126.8 1.0X +dayofyear of timestamp wholestage on 1253 1268 10 8.0 125.3 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofmonth of timestamp wholestage off 1259 1263 5 7.9 125.9 1.0X -dayofmonth of timestamp wholestage on 1201 1205 5 8.3 120.1 1.0X +dayofmonth of timestamp wholestage off 1223 1224 1 8.2 122.3 1.0X +dayofmonth of timestamp wholestage on 1231 1246 14 8.1 123.1 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofweek of timestamp wholestage off 1406 1410 6 7.1 140.6 1.0X -dayofweek of timestamp wholestage on 1387 1402 15 7.2 138.7 1.0X +dayofweek of timestamp wholestage off 1398 1406 12 7.2 139.8 1.0X +dayofweek of timestamp wholestage on 1387 1399 15 7.2 138.7 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekday of timestamp wholestage off 1355 1367 18 7.4 135.5 1.0X -weekday of timestamp wholestage on 1311 1321 10 7.6 131.1 1.0X +weekday of timestamp wholestage off 1327 1333 9 7.5 132.7 1.0X +weekday of timestamp wholestage on 1329 1333 4 7.5 132.9 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz hour of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -hour of timestamp wholestage off 996 997 2 10.0 99.6 1.0X -hour of timestamp wholestage on 930 936 6 10.7 93.0 1.1X +hour of timestamp wholestage off 1005 1016 15 9.9 100.5 1.0X +hour of timestamp wholestage on 934 940 4 10.7 93.4 1.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -minute of timestamp wholestage off 1005 1012 10 9.9 100.5 1.0X -minute of timestamp wholestage on 949 952 3 10.5 94.9 1.1X +minute of timestamp wholestage off 1003 1009 8 10.0 100.3 1.0X +minute of timestamp wholestage on 934 938 7 10.7 93.4 1.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz second of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -second of timestamp wholestage off 1013 1014 1 9.9 101.3 1.0X -second of timestamp wholestage on 933 934 2 10.7 93.3 1.1X +second of timestamp wholestage off 997 998 2 10.0 99.7 1.0X +second of timestamp wholestage on 925 935 8 10.8 92.5 1.1X ================================================================================================ @@ -124,15 +124,15 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_date wholestage off 291 293 2 34.3 29.1 1.0X -current_date wholestage on 280 284 3 35.7 28.0 1.0X +current_date wholestage off 297 297 0 33.7 29.7 1.0X +current_date wholestage on 280 282 2 35.7 28.0 1.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_timestamp wholestage off 311 324 18 32.1 31.1 1.0X -current_timestamp wholestage on 275 364 85 36.3 27.5 1.1X +current_timestamp wholestage off 307 337 43 32.6 30.7 1.0X +current_timestamp wholestage on 260 284 29 38.4 26.0 1.2X ================================================================================================ @@ -143,43 +143,43 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date wholestage off 1077 1079 3 9.3 107.7 1.0X -cast to date wholestage on 1018 1030 14 9.8 101.8 1.1X +cast to date wholestage off 1066 1073 10 9.4 106.6 1.0X +cast to date wholestage on 997 1003 6 10.0 99.7 1.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -last_day wholestage off 1257 1260 4 8.0 125.7 1.0X -last_day wholestage on 1218 1227 14 8.2 121.8 1.0X +last_day wholestage off 1238 1242 6 8.1 123.8 1.0X +last_day wholestage on 1259 1272 12 7.9 125.9 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -next_day wholestage off 1140 1141 1 8.8 114.0 1.0X -next_day wholestage on 1067 1076 11 9.4 106.7 1.1X +next_day wholestage off 1116 1138 32 9.0 111.6 1.0X +next_day wholestage on 1052 1063 11 9.5 105.2 1.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_add wholestage off 1062 1064 3 9.4 106.2 1.0X -date_add wholestage on 1046 1055 11 9.6 104.6 1.0X +date_add wholestage off 1048 1049 1 9.5 104.8 1.0X +date_add wholestage on 1035 1039 3 9.7 103.5 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_sub wholestage off 1082 1083 1 9.2 108.2 1.0X -date_sub wholestage on 1047 1056 12 9.6 104.7 1.0X +date_sub wholestage off 1119 1127 11 8.9 111.9 1.0X +date_sub wholestage on 1028 1039 7 9.7 102.8 1.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -add_months wholestage off 1430 1431 1 7.0 143.0 1.0X -add_months wholestage on 1441 1446 8 6.9 144.1 1.0X +add_months wholestage off 1421 1421 0 7.0 142.1 1.0X +add_months wholestage on 1423 1434 11 7.0 142.3 1.0X ================================================================================================ @@ -190,8 +190,8 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz format date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -format date wholestage off 5442 5549 150 1.8 544.2 1.0X -format date wholestage on 5529 5655 236 1.8 552.9 1.0X +format date wholestage off 5293 5296 5 1.9 529.3 1.0X +format date wholestage on 5143 5157 19 1.9 514.3 1.0X ================================================================================================ @@ -202,8 +202,8 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_unixtime: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_unixtime wholestage off 7416 7440 34 1.3 741.6 1.0X -from_unixtime wholestage on 7372 7391 17 1.4 737.2 1.0X +from_unixtime wholestage off 7136 7136 1 1.4 713.6 1.0X +from_unixtime wholestage on 7049 7068 29 1.4 704.9 1.0X ================================================================================================ @@ -214,15 +214,15 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_utc_timestamp wholestage off 1316 1320 6 7.6 131.6 1.0X -from_utc_timestamp wholestage on 1268 1272 4 7.9 126.8 1.0X +from_utc_timestamp wholestage off 1325 1329 6 7.5 132.5 1.0X +from_utc_timestamp wholestage on 1269 1273 4 7.9 126.9 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_utc_timestamp wholestage off 1653 1657 6 6.0 165.3 1.0X -to_utc_timestamp wholestage on 1594 1599 4 6.3 159.4 1.0X +to_utc_timestamp wholestage off 1684 1691 10 5.9 168.4 1.0X +to_utc_timestamp wholestage on 1641 1648 9 6.1 164.1 1.0X ================================================================================================ @@ -233,29 +233,29 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast interval wholestage off 341 343 3 29.4 34.1 1.0X -cast interval wholestage on 279 282 1 35.8 27.9 1.2X +cast interval wholestage off 343 346 4 29.1 34.3 1.0X +cast interval wholestage on 281 282 1 35.6 28.1 1.2X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz datediff: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -datediff wholestage off 1862 1865 4 5.4 186.2 1.0X -datediff wholestage on 1769 1783 15 5.7 176.9 1.1X +datediff wholestage off 1831 1840 13 5.5 183.1 1.0X +datediff wholestage on 1759 1769 15 5.7 175.9 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -months_between wholestage off 5594 5599 7 1.8 559.4 1.0X -months_between wholestage on 5498 5508 11 1.8 549.8 1.0X +months_between wholestage off 5729 5747 25 1.7 572.9 1.0X +months_between wholestage on 5710 5720 9 1.8 571.0 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -window wholestage off 2044 2127 117 0.5 2044.3 1.0X -window wholestage on 48057 48109 54 0.0 48056.9 0.0X +window wholestage off 2183 2189 9 0.5 2182.6 1.0X +window wholestage on 46835 46944 88 0.0 46834.8 0.0X ================================================================================================ @@ -266,134 +266,134 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YEAR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YEAR wholestage off 2540 2542 3 3.9 254.0 1.0X -date_trunc YEAR wholestage on 2486 2507 29 4.0 248.6 1.0X +date_trunc YEAR wholestage off 2668 2672 5 3.7 266.8 1.0X +date_trunc YEAR wholestage on 2719 2731 9 3.7 271.9 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YYYY wholestage off 2542 2543 3 3.9 254.2 1.0X -date_trunc YYYY wholestage on 2491 2498 9 4.0 249.1 1.0X +date_trunc YYYY wholestage off 2672 2677 8 3.7 267.2 1.0X +date_trunc YYYY wholestage on 2710 2726 12 3.7 271.0 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YY wholestage off 2545 2569 35 3.9 254.5 1.0X -date_trunc YY wholestage on 2487 2493 4 4.0 248.7 1.0X +date_trunc YY wholestage off 2670 2673 4 3.7 267.0 1.0X +date_trunc YY wholestage on 2711 2720 7 3.7 271.1 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MON wholestage off 2590 2590 1 3.9 259.0 1.0X -date_trunc MON wholestage on 2506 2520 12 4.0 250.6 1.0X +date_trunc MON wholestage off 2674 2674 0 3.7 267.4 1.0X +date_trunc MON wholestage on 2667 2677 10 3.7 266.7 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MONTH wholestage off 2595 2603 11 3.9 259.5 1.0X -date_trunc MONTH wholestage on 2505 2516 12 4.0 250.5 1.0X +date_trunc MONTH wholestage off 2675 2686 16 3.7 267.5 1.0X +date_trunc MONTH wholestage on 2667 2674 6 3.7 266.7 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MM wholestage off 2605 2612 10 3.8 260.5 1.0X -date_trunc MM wholestage on 2501 2515 11 4.0 250.1 1.0X +date_trunc MM wholestage off 2673 2674 1 3.7 267.3 1.0X +date_trunc MM wholestage on 2664 2669 4 3.8 266.4 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DAY wholestage off 2225 2229 5 4.5 222.5 1.0X -date_trunc DAY wholestage on 2184 2196 9 4.6 218.4 1.0X +date_trunc DAY wholestage off 2281 2288 10 4.4 228.1 1.0X +date_trunc DAY wholestage on 2302 2312 8 4.3 230.2 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DD wholestage off 2232 2236 6 4.5 223.2 1.0X -date_trunc DD wholestage on 2183 2190 6 4.6 218.3 1.0X +date_trunc DD wholestage off 2281 2283 3 4.4 228.1 1.0X +date_trunc DD wholestage on 2291 2302 11 4.4 229.1 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc HOUR wholestage off 2194 2199 7 4.6 219.4 1.0X -date_trunc HOUR wholestage on 2160 2166 5 4.6 216.0 1.0X +date_trunc HOUR wholestage off 2331 2332 1 4.3 233.1 1.0X +date_trunc HOUR wholestage on 2290 2304 11 4.4 229.0 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MINUTE wholestage off 390 396 9 25.7 39.0 1.0X -date_trunc MINUTE wholestage on 331 337 7 30.2 33.1 1.2X +date_trunc MINUTE wholestage off 379 385 9 26.4 37.9 1.0X +date_trunc MINUTE wholestage on 371 376 5 27.0 37.1 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc SECOND: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc SECOND wholestage off 375 381 8 26.7 37.5 1.0X -date_trunc SECOND wholestage on 332 346 14 30.1 33.2 1.1X +date_trunc SECOND wholestage off 375 376 1 26.7 37.5 1.0X +date_trunc SECOND wholestage on 370 376 8 27.0 37.0 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc WEEK: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc WEEK wholestage off 2439 2443 6 4.1 243.9 1.0X -date_trunc WEEK wholestage on 2390 2409 32 4.2 239.0 1.0X +date_trunc WEEK wholestage off 2597 2604 10 3.9 259.7 1.0X +date_trunc WEEK wholestage on 2591 2605 13 3.9 259.1 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc QUARTER wholestage off 3290 3292 4 3.0 329.0 1.0X -date_trunc QUARTER wholestage on 3214 3218 3 3.1 321.4 1.0X +date_trunc QUARTER wholestage off 3501 3511 14 2.9 350.1 1.0X +date_trunc QUARTER wholestage on 3477 3489 9 2.9 347.7 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc year wholestage off 308 310 3 32.5 30.8 1.0X -trunc year wholestage on 289 293 6 34.7 28.9 1.1X +trunc year wholestage off 332 334 3 30.1 33.2 1.0X +trunc year wholestage on 332 346 17 30.1 33.2 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yyyy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yyyy wholestage off 309 311 3 32.4 30.9 1.0X -trunc yyyy wholestage on 289 294 7 34.6 28.9 1.1X +trunc yyyy wholestage off 331 331 0 30.2 33.1 1.0X +trunc yyyy wholestage on 336 339 4 29.8 33.6 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yy wholestage off 311 311 0 32.2 31.1 1.0X -trunc yy wholestage on 288 294 7 34.7 28.8 1.1X +trunc yy wholestage off 330 342 17 30.3 33.0 1.0X +trunc yy wholestage on 333 337 3 30.0 33.3 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mon: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mon wholestage off 313 313 0 32.0 31.3 1.0X -trunc mon wholestage on 287 290 2 34.8 28.7 1.1X +trunc mon wholestage off 334 335 1 30.0 33.4 1.0X +trunc mon wholestage on 333 347 9 30.0 33.3 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc month: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc month wholestage off 310 310 0 32.3 31.0 1.0X -trunc month wholestage on 287 290 2 34.8 28.7 1.1X +trunc month wholestage off 332 333 1 30.1 33.2 1.0X +trunc month wholestage on 333 340 7 30.0 33.3 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mm: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mm wholestage off 311 312 1 32.1 31.1 1.0X -trunc mm wholestage on 287 296 9 34.8 28.7 1.1X +trunc mm wholestage off 328 336 11 30.5 32.8 1.0X +trunc mm wholestage on 333 343 11 30.0 33.3 1.0X ================================================================================================ @@ -404,36 +404,36 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to timestamp str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to timestamp str wholestage off 169 170 1 5.9 168.9 1.0X -to timestamp str wholestage on 161 168 11 6.2 161.0 1.0X +to timestamp str wholestage off 170 171 1 5.9 170.1 1.0X +to timestamp str wholestage on 172 174 2 5.8 171.6 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_timestamp wholestage off 1360 1361 1 0.7 1359.6 1.0X -to_timestamp wholestage on 1362 1366 6 0.7 1362.0 1.0X +to_timestamp wholestage off 1437 1439 3 0.7 1437.0 1.0X +to_timestamp wholestage on 1288 1292 5 0.8 1288.1 1.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_unix_timestamp wholestage off 1343 1346 4 0.7 1342.6 1.0X -to_unix_timestamp wholestage on 1356 1359 2 0.7 1356.2 1.0X +to_unix_timestamp wholestage off 1352 1353 2 0.7 1352.0 1.0X +to_unix_timestamp wholestage on 1314 1319 5 0.8 1314.4 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to date str wholestage off 227 230 4 4.4 227.0 1.0X -to date str wholestage on 299 302 3 3.3 299.0 0.8X +to date str wholestage off 211 215 6 4.7 210.7 1.0X +to date str wholestage on 217 217 1 4.6 216.5 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_date wholestage off 3413 3440 38 0.3 3413.0 1.0X -to_date wholestage on 3392 3402 12 0.3 3392.3 1.0X +to_date wholestage off 3281 3295 20 0.3 3280.9 1.0X +to_date wholestage on 3223 3239 17 0.3 3222.8 1.0X ================================================================================================ @@ -444,14 +444,14 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz To/from Java's date-time: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -From java.sql.Date 410 415 7 12.2 82.0 1.0X -From java.time.LocalDate 332 333 1 15.1 66.4 1.2X -Collect java.sql.Date 1891 2542 829 2.6 378.1 0.2X -Collect java.time.LocalDate 1630 2138 441 3.1 326.0 0.3X -From java.sql.Timestamp 254 259 6 19.7 50.9 1.6X -From java.time.Instant 302 306 4 16.6 60.3 1.4X -Collect longs 1134 1265 117 4.4 226.8 0.4X -Collect java.sql.Timestamp 1441 1458 16 3.5 288.1 0.3X -Collect java.time.Instant 1680 1928 253 3.0 336.0 0.2X +From java.sql.Date 446 447 1 11.2 89.1 1.0X +From java.time.LocalDate 354 356 1 14.1 70.8 1.3X +Collect java.sql.Date 2722 3091 495 1.8 544.4 0.2X +Collect java.time.LocalDate 1786 1836 60 2.8 357.2 0.2X +From java.sql.Timestamp 275 287 19 18.2 55.0 1.6X +From java.time.Instant 325 328 3 15.4 65.0 1.4X +Collect longs 1300 1321 25 3.8 260.0 0.3X +Collect java.sql.Timestamp 1450 1557 102 3.4 290.0 0.3X +Collect java.time.Instant 1499 1599 87 3.3 299.9 0.3X diff --git a/sql/core/benchmarks/DateTimeBenchmark-results.txt b/sql/core/benchmarks/DateTimeBenchmark-results.txt index 7586295778bd8..7a9aa4badfeb7 100644 --- a/sql/core/benchmarks/DateTimeBenchmark-results.txt +++ b/sql/core/benchmarks/DateTimeBenchmark-results.txt @@ -6,18 +6,18 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz datetime +/- interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date + interval(m) 1638 1701 89 6.1 163.8 1.0X -date + interval(m, d) 1785 1790 7 5.6 178.5 0.9X -date + interval(m, d, ms) 6229 6270 58 1.6 622.9 0.3X -date - interval(m) 1500 1503 4 6.7 150.0 1.1X -date - interval(m, d) 1764 1766 3 5.7 176.4 0.9X -date - interval(m, d, ms) 6428 6446 25 1.6 642.8 0.3X -timestamp + interval(m) 2719 2722 4 3.7 271.9 0.6X -timestamp + interval(m, d) 3011 3021 14 3.3 301.1 0.5X -timestamp + interval(m, d, ms) 3405 3412 9 2.9 340.5 0.5X -timestamp - interval(m) 2759 2764 7 3.6 275.9 0.6X -timestamp - interval(m, d) 3094 3112 25 3.2 309.4 0.5X -timestamp - interval(m, d, ms) 3388 3392 5 3.0 338.8 0.5X +date + interval(m) 1555 1634 113 6.4 155.5 1.0X +date + interval(m, d) 1774 1797 33 5.6 177.4 0.9X +date + interval(m, d, ms) 6293 6335 59 1.6 629.3 0.2X +date - interval(m) 1461 1468 10 6.8 146.1 1.1X +date - interval(m, d) 1741 1741 0 5.7 174.1 0.9X +date - interval(m, d, ms) 6503 6518 21 1.5 650.3 0.2X +timestamp + interval(m) 2384 2385 1 4.2 238.4 0.7X +timestamp + interval(m, d) 2683 2684 2 3.7 268.3 0.6X +timestamp + interval(m, d, ms) 2987 3001 19 3.3 298.7 0.5X +timestamp - interval(m) 2391 2395 5 4.2 239.1 0.7X +timestamp - interval(m, d) 2674 2684 14 3.7 267.4 0.6X +timestamp - interval(m, d, ms) 3005 3007 3 3.3 300.5 0.5X ================================================================================================ @@ -28,92 +28,92 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp wholestage off 319 323 6 31.4 31.9 1.0X -cast to timestamp wholestage on 304 311 8 32.9 30.4 1.0X +cast to timestamp wholestage off 313 320 10 31.9 31.3 1.0X +cast to timestamp wholestage on 325 341 18 30.8 32.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -year of timestamp wholestage off 1234 1239 6 8.1 123.4 1.0X -year of timestamp wholestage on 1229 1244 22 8.1 122.9 1.0X +year of timestamp wholestage off 1216 1216 1 8.2 121.6 1.0X +year of timestamp wholestage on 1226 1243 13 8.2 122.6 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -quarter of timestamp wholestage off 1440 1445 7 6.9 144.0 1.0X -quarter of timestamp wholestage on 1358 1361 3 7.4 135.8 1.1X +quarter of timestamp wholestage off 1417 1421 5 7.1 141.7 1.0X +quarter of timestamp wholestage on 1358 1365 8 7.4 135.8 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -month of timestamp wholestage off 1239 1240 1 8.1 123.9 1.0X -month of timestamp wholestage on 1221 1239 26 8.2 122.1 1.0X +month of timestamp wholestage off 1219 1220 1 8.2 121.9 1.0X +month of timestamp wholestage on 1222 1227 7 8.2 122.2 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekofyear of timestamp wholestage off 1926 1934 11 5.2 192.6 1.0X -weekofyear of timestamp wholestage on 1901 1911 10 5.3 190.1 1.0X +weekofyear of timestamp wholestage off 1950 1950 0 5.1 195.0 1.0X +weekofyear of timestamp wholestage on 1890 1899 8 5.3 189.0 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -day of timestamp wholestage off 1225 1229 6 8.2 122.5 1.0X -day of timestamp wholestage on 1217 1225 7 8.2 121.7 1.0X +day of timestamp wholestage off 1212 1213 2 8.3 121.2 1.0X +day of timestamp wholestage on 1216 1227 13 8.2 121.6 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofyear of timestamp wholestage off 1290 1295 7 7.8 129.0 1.0X -dayofyear of timestamp wholestage on 1262 1270 7 7.9 126.2 1.0X +dayofyear of timestamp wholestage off 1282 1284 3 7.8 128.2 1.0X +dayofyear of timestamp wholestage on 1269 1274 5 7.9 126.9 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofmonth of timestamp wholestage off 1239 1239 1 8.1 123.9 1.0X -dayofmonth of timestamp wholestage on 1215 1222 8 8.2 121.5 1.0X +dayofmonth of timestamp wholestage off 1214 1219 7 8.2 121.4 1.0X +dayofmonth of timestamp wholestage on 1216 1224 6 8.2 121.6 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofweek of timestamp wholestage off 1421 1422 2 7.0 142.1 1.0X -dayofweek of timestamp wholestage on 1379 1388 8 7.3 137.9 1.0X +dayofweek of timestamp wholestage off 1403 1430 39 7.1 140.3 1.0X +dayofweek of timestamp wholestage on 1378 1386 8 7.3 137.8 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekday of timestamp wholestage off 1349 1351 2 7.4 134.9 1.0X -weekday of timestamp wholestage on 1320 1327 8 7.6 132.0 1.0X +weekday of timestamp wholestage off 1344 1353 13 7.4 134.4 1.0X +weekday of timestamp wholestage on 1316 1322 5 7.6 131.6 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz hour of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -hour of timestamp wholestage off 1024 1024 0 9.8 102.4 1.0X -hour of timestamp wholestage on 921 929 11 10.9 92.1 1.1X +hour of timestamp wholestage off 992 1000 10 10.1 99.2 1.0X +hour of timestamp wholestage on 960 962 3 10.4 96.0 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -minute of timestamp wholestage off 977 982 6 10.2 97.7 1.0X -minute of timestamp wholestage on 927 929 2 10.8 92.7 1.1X +minute of timestamp wholestage off 989 1000 16 10.1 98.9 1.0X +minute of timestamp wholestage on 965 974 13 10.4 96.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz second of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -second of timestamp wholestage off 987 989 3 10.1 98.7 1.0X -second of timestamp wholestage on 923 926 5 10.8 92.3 1.1X +second of timestamp wholestage off 974 977 5 10.3 97.4 1.0X +second of timestamp wholestage on 959 966 8 10.4 95.9 1.0X ================================================================================================ @@ -124,15 +124,15 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_date wholestage off 303 311 12 33.0 30.3 1.0X -current_date wholestage on 266 271 5 37.5 26.6 1.1X +current_date wholestage off 281 282 2 35.6 28.1 1.0X +current_date wholestage on 294 300 5 34.0 29.4 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_timestamp wholestage off 297 297 1 33.7 29.7 1.0X -current_timestamp wholestage on 264 272 7 37.8 26.4 1.1X +current_timestamp wholestage off 282 296 19 35.4 28.2 1.0X +current_timestamp wholestage on 304 331 31 32.9 30.4 0.9X ================================================================================================ @@ -143,43 +143,43 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date wholestage off 1062 1063 2 9.4 106.2 1.0X -cast to date wholestage on 1007 1021 20 9.9 100.7 1.1X +cast to date wholestage off 1060 1061 1 9.4 106.0 1.0X +cast to date wholestage on 1021 1026 10 9.8 102.1 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -last_day wholestage off 1262 1265 5 7.9 126.2 1.0X -last_day wholestage on 1244 1256 14 8.0 124.4 1.0X +last_day wholestage off 1278 1280 3 7.8 127.8 1.0X +last_day wholestage on 1560 1566 6 6.4 156.0 0.8X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -next_day wholestage off 1119 1121 2 8.9 111.9 1.0X -next_day wholestage on 1057 1063 6 9.5 105.7 1.1X +next_day wholestage off 1091 1093 3 9.2 109.1 1.0X +next_day wholestage on 1070 1076 9 9.3 107.0 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_add wholestage off 1054 1059 7 9.5 105.4 1.0X -date_add wholestage on 1037 1069 52 9.6 103.7 1.0X +date_add wholestage off 1041 1047 8 9.6 104.1 1.0X +date_add wholestage on 1044 1050 4 9.6 104.4 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_sub wholestage off 1054 1056 4 9.5 105.4 1.0X -date_sub wholestage on 1036 1040 4 9.7 103.6 1.0X +date_sub wholestage off 1038 1040 3 9.6 103.8 1.0X +date_sub wholestage on 1057 1061 4 9.5 105.7 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -add_months wholestage off 1408 1421 19 7.1 140.8 1.0X -add_months wholestage on 1434 1440 7 7.0 143.4 1.0X +add_months wholestage off 1401 1401 1 7.1 140.1 1.0X +add_months wholestage on 1438 1442 4 7.0 143.8 1.0X ================================================================================================ @@ -190,8 +190,8 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz format date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -format date wholestage off 5937 6169 328 1.7 593.7 1.0X -format date wholestage on 5836 5878 74 1.7 583.6 1.0X +format date wholestage off 5482 5803 454 1.8 548.2 1.0X +format date wholestage on 5502 5518 9 1.8 550.2 1.0X ================================================================================================ @@ -202,8 +202,8 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_unixtime: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_unixtime wholestage off 8904 8914 14 1.1 890.4 1.0X -from_unixtime wholestage on 8918 8936 13 1.1 891.8 1.0X +from_unixtime wholestage off 8538 8553 22 1.2 853.8 1.0X +from_unixtime wholestage on 8545 8552 6 1.2 854.5 1.0X ================================================================================================ @@ -214,15 +214,15 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_utc_timestamp wholestage off 1110 1112 3 9.0 111.0 1.0X -from_utc_timestamp wholestage on 1115 1119 3 9.0 111.5 1.0X +from_utc_timestamp wholestage off 1094 1099 8 9.1 109.4 1.0X +from_utc_timestamp wholestage on 1109 1114 5 9.0 110.9 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_utc_timestamp wholestage off 1524 1525 1 6.6 152.4 1.0X -to_utc_timestamp wholestage on 1450 1458 14 6.9 145.0 1.1X +to_utc_timestamp wholestage off 1466 1469 4 6.8 146.6 1.0X +to_utc_timestamp wholestage on 1401 1408 7 7.1 140.1 1.0X ================================================================================================ @@ -233,29 +233,29 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast interval wholestage off 341 342 1 29.3 34.1 1.0X -cast interval wholestage on 285 294 7 35.1 28.5 1.2X +cast interval wholestage off 332 332 0 30.1 33.2 1.0X +cast interval wholestage on 315 324 10 31.7 31.5 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz datediff: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -datediff wholestage off 1874 1881 10 5.3 187.4 1.0X -datediff wholestage on 1785 1791 3 5.6 178.5 1.0X +datediff wholestage off 1796 1802 8 5.6 179.6 1.0X +datediff wholestage on 1758 1764 10 5.7 175.8 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -months_between wholestage off 5038 5042 5 2.0 503.8 1.0X -months_between wholestage on 4979 4987 8 2.0 497.9 1.0X +months_between wholestage off 4833 4836 4 2.1 483.3 1.0X +months_between wholestage on 4777 4780 2 2.1 477.7 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -window wholestage off 1716 1841 177 0.6 1716.2 1.0X -window wholestage on 46024 46063 27 0.0 46024.1 0.0X +window wholestage off 1812 1908 136 0.6 1811.7 1.0X +window wholestage on 46279 46376 74 0.0 46278.8 0.0X ================================================================================================ @@ -266,134 +266,134 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YEAR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YEAR wholestage off 2428 2429 2 4.1 242.8 1.0X -date_trunc YEAR wholestage on 2451 2469 12 4.1 245.1 1.0X +date_trunc YEAR wholestage off 2367 2368 1 4.2 236.7 1.0X +date_trunc YEAR wholestage on 2321 2334 22 4.3 232.1 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YYYY wholestage off 2423 2426 3 4.1 242.3 1.0X -date_trunc YYYY wholestage on 2454 2462 8 4.1 245.4 1.0X +date_trunc YYYY wholestage off 2330 2334 5 4.3 233.0 1.0X +date_trunc YYYY wholestage on 2326 2332 5 4.3 232.6 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YY wholestage off 2421 2441 28 4.1 242.1 1.0X -date_trunc YY wholestage on 2453 2461 9 4.1 245.3 1.0X +date_trunc YY wholestage off 2334 2335 1 4.3 233.4 1.0X +date_trunc YY wholestage on 2315 2324 6 4.3 231.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MON wholestage off 2425 2427 3 4.1 242.5 1.0X -date_trunc MON wholestage on 2431 2438 9 4.1 243.1 1.0X +date_trunc MON wholestage off 2327 2330 4 4.3 232.7 1.0X +date_trunc MON wholestage on 2279 2289 12 4.4 227.9 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MONTH wholestage off 2427 2433 8 4.1 242.7 1.0X -date_trunc MONTH wholestage on 2429 2435 4 4.1 242.9 1.0X +date_trunc MONTH wholestage off 2330 2332 2 4.3 233.0 1.0X +date_trunc MONTH wholestage on 2277 2284 6 4.4 227.7 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MM wholestage off 2425 2431 9 4.1 242.5 1.0X -date_trunc MM wholestage on 2430 2435 4 4.1 243.0 1.0X +date_trunc MM wholestage off 2328 2329 2 4.3 232.8 1.0X +date_trunc MM wholestage on 2279 2284 4 4.4 227.9 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DAY wholestage off 2117 2119 4 4.7 211.7 1.0X -date_trunc DAY wholestage on 2036 2118 174 4.9 203.6 1.0X +date_trunc DAY wholestage off 1974 1984 14 5.1 197.4 1.0X +date_trunc DAY wholestage on 1914 1922 7 5.2 191.4 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DD wholestage off 2116 2119 5 4.7 211.6 1.0X -date_trunc DD wholestage on 2035 2043 10 4.9 203.5 1.0X +date_trunc DD wholestage off 1967 1976 12 5.1 196.7 1.0X +date_trunc DD wholestage on 1913 1917 4 5.2 191.3 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc HOUR wholestage off 2013 2014 2 5.0 201.3 1.0X -date_trunc HOUR wholestage on 2077 2088 13 4.8 207.7 1.0X +date_trunc HOUR wholestage off 1970 1970 0 5.1 197.0 1.0X +date_trunc HOUR wholestage on 1945 1946 2 5.1 194.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MINUTE wholestage off 363 368 8 27.6 36.3 1.0X -date_trunc MINUTE wholestage on 321 326 7 31.2 32.1 1.1X +date_trunc MINUTE wholestage off 361 361 1 27.7 36.1 1.0X +date_trunc MINUTE wholestage on 331 336 4 30.2 33.1 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc SECOND: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc SECOND wholestage off 365 366 0 27.4 36.5 1.0X -date_trunc SECOND wholestage on 319 332 16 31.4 31.9 1.1X +date_trunc SECOND wholestage off 360 361 1 27.8 36.0 1.0X +date_trunc SECOND wholestage on 335 348 15 29.8 33.5 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc WEEK: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc WEEK wholestage off 2371 2376 7 4.2 237.1 1.0X -date_trunc WEEK wholestage on 2314 2322 8 4.3 231.4 1.0X +date_trunc WEEK wholestage off 2232 2236 6 4.5 223.2 1.0X +date_trunc WEEK wholestage on 2225 2232 6 4.5 222.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc QUARTER wholestage off 3334 3335 1 3.0 333.4 1.0X -date_trunc QUARTER wholestage on 3286 3291 7 3.0 328.6 1.0X +date_trunc QUARTER wholestage off 3083 3086 4 3.2 308.3 1.0X +date_trunc QUARTER wholestage on 3073 3086 16 3.3 307.3 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc year wholestage off 303 304 2 33.0 30.3 1.0X -trunc year wholestage on 283 291 5 35.3 28.3 1.1X +trunc year wholestage off 321 321 0 31.1 32.1 1.0X +trunc year wholestage on 299 303 5 33.5 29.9 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yyyy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yyyy wholestage off 324 330 8 30.9 32.4 1.0X -trunc yyyy wholestage on 283 291 9 35.3 28.3 1.1X +trunc yyyy wholestage off 323 327 5 30.9 32.3 1.0X +trunc yyyy wholestage on 299 302 3 33.4 29.9 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yy wholestage off 304 305 3 32.9 30.4 1.0X -trunc yy wholestage on 283 302 28 35.3 28.3 1.1X +trunc yy wholestage off 315 315 1 31.8 31.5 1.0X +trunc yy wholestage on 299 304 4 33.4 29.9 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mon: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mon wholestage off 315 319 6 31.7 31.5 1.0X -trunc mon wholestage on 284 287 5 35.3 28.4 1.1X +trunc mon wholestage off 320 321 1 31.2 32.0 1.0X +trunc mon wholestage on 299 307 10 33.4 29.9 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc month: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc month wholestage off 305 314 13 32.8 30.5 1.0X -trunc month wholestage on 283 292 14 35.3 28.3 1.1X +trunc month wholestage off 316 317 1 31.6 31.6 1.0X +trunc month wholestage on 299 302 5 33.5 29.9 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mm: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mm wholestage off 301 301 0 33.2 30.1 1.0X -trunc mm wholestage on 285 290 7 35.1 28.5 1.1X +trunc mm wholestage off 313 313 1 32.0 31.3 1.0X +trunc mm wholestage on 298 302 4 33.5 29.8 1.0X ================================================================================================ @@ -404,36 +404,36 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to timestamp str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to timestamp str wholestage off 218 220 3 4.6 218.4 1.0X -to timestamp str wholestage on 213 216 6 4.7 212.5 1.0X +to timestamp str wholestage off 217 217 0 4.6 217.3 1.0X +to timestamp str wholestage on 209 212 2 4.8 209.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_timestamp wholestage off 1838 1842 5 0.5 1838.1 1.0X -to_timestamp wholestage on 1952 1971 11 0.5 1952.2 0.9X +to_timestamp wholestage off 1676 1677 2 0.6 1675.6 1.0X +to_timestamp wholestage on 1599 1606 8 0.6 1599.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_unix_timestamp wholestage off 1987 1988 1 0.5 1986.9 1.0X -to_unix_timestamp wholestage on 1944 1948 3 0.5 1944.2 1.0X +to_unix_timestamp wholestage off 1582 1589 9 0.6 1582.1 1.0X +to_unix_timestamp wholestage on 1634 1637 3 0.6 1633.8 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to date str wholestage off 263 264 0 3.8 263.5 1.0X -to date str wholestage on 263 265 2 3.8 262.6 1.0X +to date str wholestage off 275 282 9 3.6 275.0 1.0X +to date str wholestage on 264 265 2 3.8 263.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_date wholestage off 3560 3567 11 0.3 3559.7 1.0X -to_date wholestage on 3525 3534 10 0.3 3524.8 1.0X +to_date wholestage off 3170 3188 25 0.3 3170.1 1.0X +to_date wholestage on 3134 3143 10 0.3 3134.3 1.0X ================================================================================================ @@ -444,14 +444,14 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz To/from Java's date-time: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -From java.sql.Date 405 416 16 12.3 81.0 1.0X -From java.time.LocalDate 344 352 14 14.5 68.8 1.2X -Collect java.sql.Date 1622 2553 1372 3.1 324.4 0.2X -Collect java.time.LocalDate 1464 1482 20 3.4 292.8 0.3X -From java.sql.Timestamp 248 258 15 20.2 49.6 1.6X -From java.time.Instant 237 243 7 21.1 47.4 1.7X -Collect longs 1252 1341 109 4.0 250.5 0.3X -Collect java.sql.Timestamp 1515 1516 2 3.3 302.9 0.3X -Collect java.time.Instant 1379 1490 96 3.6 275.8 0.3X +From java.sql.Date 407 413 7 12.3 81.5 1.0X +From java.time.LocalDate 340 344 5 14.7 68.1 1.2X +Collect java.sql.Date 1700 2658 1422 2.9 340.0 0.2X +Collect java.time.LocalDate 1473 1494 30 3.4 294.6 0.3X +From java.sql.Timestamp 252 266 13 19.8 50.5 1.6X +From java.time.Instant 236 243 7 21.1 47.3 1.7X +Collect longs 1280 1337 79 3.9 256.1 0.3X +Collect java.sql.Timestamp 1485 1501 15 3.4 297.0 0.3X +Collect java.time.Instant 1441 1465 37 3.5 288.1 0.3X diff --git a/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt b/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt index 03bc334471e56..d0cd591da4c94 100644 --- a/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt @@ -3,110 +3,110 @@ Benchmark for performance of JSON parsing ================================================================================================ Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 46010 46118 113 2.2 460.1 1.0X -UTF-8 is set 54407 55427 1718 1.8 544.1 0.8X +No encoding 68879 68993 116 1.5 688.8 1.0X +UTF-8 is set 115270 115602 455 0.9 1152.7 0.6X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 26614 28220 1461 3.8 266.1 1.0X -UTF-8 is set 42765 43400 550 2.3 427.6 0.6X +No encoding 47452 47538 113 2.1 474.5 1.0X +UTF-8 is set 77330 77354 30 1.3 773.3 0.6X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 35696 35821 113 0.3 3569.6 1.0X -UTF-8 is set 55441 56176 1037 0.2 5544.1 0.6X +No encoding 60470 60900 534 0.2 6047.0 1.0X +UTF-8 is set 104733 104931 189 0.1 10473.3 0.6X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 61514 62968 NaN 0.0 123027.2 1.0X -UTF-8 is set 72096 72933 1162 0.0 144192.7 0.9X +No encoding 130302 131072 976 0.0 260604.6 1.0X +UTF-8 is set 150860 151284 377 0.0 301720.1 0.9X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns 9859 9913 79 1.0 985.9 1.0X -Select 1 column 10981 11003 36 0.9 1098.1 0.9X +Select 10 columns 18619 18684 99 0.5 1861.9 1.0X +Select 1 column 24227 24270 38 0.4 2422.7 0.8X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz creation of JSON parser per line: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Short column without encoding 3555 3579 27 2.8 355.5 1.0X -Short column with UTF-8 5204 5227 35 1.9 520.4 0.7X -Wide column without encoding 60458 60637 164 0.2 6045.8 0.1X -Wide column with UTF-8 77544 78111 551 0.1 7754.4 0.0X +Short column without encoding 7947 7971 21 1.3 794.7 1.0X +Short column with UTF-8 12700 12753 58 0.8 1270.0 0.6X +Wide column without encoding 92632 92955 463 0.1 9263.2 0.1X +Wide column with UTF-8 147013 147170 188 0.1 14701.3 0.1X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 342 346 3 29.2 34.2 1.0X -from_json 7123 7318 179 1.4 712.3 0.0X -json_tuple 9843 9957 132 1.0 984.3 0.0X -get_json_object 7827 8046 194 1.3 782.7 0.0X +Text read 713 734 19 14.0 71.3 1.0X +from_json 22019 22429 456 0.5 2201.9 0.0X +json_tuple 27987 28047 74 0.4 2798.7 0.0X +get_json_object 21468 21870 350 0.5 2146.8 0.0X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 1856 1884 32 26.9 37.1 1.0X -schema inferring 16734 16900 153 3.0 334.7 0.1X -parsing 14884 15203 470 3.4 297.7 0.1X +Text read 2887 2910 24 17.3 57.7 1.0X +schema inferring 31793 31843 43 1.6 635.9 0.1X +parsing 36791 37104 294 1.4 735.8 0.1X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 5932 6148 228 8.4 118.6 1.0X -Schema inferring 20836 21938 1086 2.4 416.7 0.3X -Parsing without charset 18134 18661 457 2.8 362.7 0.3X -Parsing with UTF-8 27734 28069 378 1.8 554.7 0.2X +Text read 10570 10611 45 4.7 211.4 1.0X +Schema inferring 48729 48763 41 1.0 974.6 0.2X +Parsing without charset 35490 35648 141 1.4 709.8 0.3X +Parsing with UTF-8 63853 63994 163 0.8 1277.1 0.2X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 889 914 28 11.2 88.9 1.0X -to_json(timestamp) 7920 8172 353 1.3 792.0 0.1X -write timestamps to files 6726 6822 129 1.5 672.6 0.1X -Create a dataset of dates 953 963 12 10.5 95.3 0.9X -to_json(date) 5370 5705 320 1.9 537.0 0.2X -write dates to files 4109 4166 52 2.4 410.9 0.2X +Create a dataset of timestamps 2187 2190 5 4.6 218.7 1.0X +to_json(timestamp) 16262 16503 323 0.6 1626.2 0.1X +write timestamps to files 11679 11692 12 0.9 1167.9 0.2X +Create a dataset of dates 2297 2310 12 4.4 229.7 1.0X +to_json(date) 10904 10956 46 0.9 1090.4 0.2X +write dates to files 6610 6645 35 1.5 661.0 0.3X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 1614 1675 55 6.2 161.4 1.0X -read timestamps from files 16640 16858 209 0.6 1664.0 0.1X -infer timestamps from files 33239 33388 227 0.3 3323.9 0.0X -read date text from files 1310 1340 44 7.6 131.0 1.2X -read date from files 9470 9513 41 1.1 947.0 0.2X -timestamp strings 1303 1342 47 7.7 130.3 1.2X -parse timestamps from Dataset[String] 17650 18073 380 0.6 1765.0 0.1X -infer timestamps from Dataset[String] 32623 34065 1330 0.3 3262.3 0.0X -date strings 1864 1871 7 5.4 186.4 0.9X -parse dates from Dataset[String] 10914 11316 482 0.9 1091.4 0.1X -from_json(timestamp) 21102 21990 929 0.5 2110.2 0.1X -from_json(date) 15275 15961 598 0.7 1527.5 0.1X +read timestamp text from files 2524 2530 9 4.0 252.4 1.0X +read timestamps from files 41002 41052 59 0.2 4100.2 0.1X +infer timestamps from files 84621 84939 526 0.1 8462.1 0.0X +read date text from files 2292 2302 9 4.4 229.2 1.1X +read date from files 16954 16976 21 0.6 1695.4 0.1X +timestamp strings 3067 3077 13 3.3 306.7 0.8X +parse timestamps from Dataset[String] 48690 48971 243 0.2 4869.0 0.1X +infer timestamps from Dataset[String] 97463 97786 338 0.1 9746.3 0.0X +date strings 3952 3956 3 2.5 395.2 0.6X +parse dates from Dataset[String] 24210 24241 30 0.4 2421.0 0.1X +from_json(timestamp) 71710 72242 629 0.1 7171.0 0.0X +from_json(date) 42465 42481 13 0.2 4246.5 0.1X diff --git a/sql/core/benchmarks/JsonBenchmark-results.txt b/sql/core/benchmarks/JsonBenchmark-results.txt index 0f188c4cdea56..46d2410fb47c3 100644 --- a/sql/core/benchmarks/JsonBenchmark-results.txt +++ b/sql/core/benchmarks/JsonBenchmark-results.txt @@ -3,110 +3,110 @@ Benchmark for performance of JSON parsing ================================================================================================ Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 38998 41002 NaN 2.6 390.0 1.0X -UTF-8 is set 61231 63282 1854 1.6 612.3 0.6X +No encoding 63981 64044 56 1.6 639.8 1.0X +UTF-8 is set 112672 113350 962 0.9 1126.7 0.6X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 28272 28338 70 3.5 282.7 1.0X -UTF-8 is set 58681 62243 1517 1.7 586.8 0.5X +No encoding 51256 51449 180 2.0 512.6 1.0X +UTF-8 is set 83694 83859 148 1.2 836.9 0.6X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 44026 51829 1329 0.2 4402.6 1.0X -UTF-8 is set 65839 68596 500 0.2 6583.9 0.7X +No encoding 58440 59097 569 0.2 5844.0 1.0X +UTF-8 is set 102746 102883 198 0.1 10274.6 0.6X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 72144 74820 NaN 0.0 144287.6 1.0X -UTF-8 is set 69571 77888 NaN 0.0 139142.3 1.0X +No encoding 128982 129304 356 0.0 257965.0 1.0X +UTF-8 is set 147247 147415 231 0.0 294494.1 0.9X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns 9502 9604 106 1.1 950.2 1.0X -Select 1 column 11861 11948 109 0.8 1186.1 0.8X +Select 10 columns 18837 19048 331 0.5 1883.7 1.0X +Select 1 column 24707 24723 14 0.4 2470.7 0.8X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz creation of JSON parser per line: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Short column without encoding 3830 3846 15 2.6 383.0 1.0X -Short column with UTF-8 5538 5543 7 1.8 553.8 0.7X -Wide column without encoding 66899 69158 NaN 0.1 6689.9 0.1X -Wide column with UTF-8 90052 93235 NaN 0.1 9005.2 0.0X +Short column without encoding 8218 8234 17 1.2 821.8 1.0X +Short column with UTF-8 12374 12438 107 0.8 1237.4 0.7X +Wide column without encoding 136918 137298 345 0.1 13691.8 0.1X +Wide column with UTF-8 176961 177142 257 0.1 17696.1 0.0X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 659 674 13 15.2 65.9 1.0X -from_json 7676 7943 405 1.3 767.6 0.1X -json_tuple 9881 10172 273 1.0 988.1 0.1X -get_json_object 7949 8055 119 1.3 794.9 0.1X +Text read 1268 1278 12 7.9 126.8 1.0X +from_json 23348 23479 176 0.4 2334.8 0.1X +json_tuple 29606 30221 1024 0.3 2960.6 0.0X +get_json_object 21898 22148 226 0.5 2189.8 0.1X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 3314 3326 17 15.1 66.3 1.0X -schema inferring 16549 17037 484 3.0 331.0 0.2X -parsing 15138 15283 172 3.3 302.8 0.2X +Text read 5887 5944 49 8.5 117.7 1.0X +schema inferring 46696 47054 312 1.1 933.9 0.1X +parsing 32336 32450 129 1.5 646.7 0.2X Preparing data for benchmarking ... -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 5136 5446 268 9.7 102.7 1.0X -Schema inferring 19864 20568 1191 2.5 397.3 0.3X -Parsing without charset 17535 17888 329 2.9 350.7 0.3X -Parsing with UTF-8 25609 25758 218 2.0 512.2 0.2X +Text read 9756 9769 11 5.1 195.1 1.0X +Schema inferring 51318 51433 108 1.0 1026.4 0.2X +Parsing without charset 43609 43743 118 1.1 872.2 0.2X +Parsing with UTF-8 60775 60844 106 0.8 1215.5 0.2X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 784 790 7 12.8 78.4 1.0X -to_json(timestamp) 8005 8055 50 1.2 800.5 0.1X -write timestamps to files 6515 6559 45 1.5 651.5 0.1X -Create a dataset of dates 854 881 24 11.7 85.4 0.9X -to_json(date) 5187 5194 7 1.9 518.7 0.2X -write dates to files 3663 3684 22 2.7 366.3 0.2X +Create a dataset of timestamps 1998 2015 17 5.0 199.8 1.0X +to_json(timestamp) 18156 18317 263 0.6 1815.6 0.1X +write timestamps to files 12912 12917 5 0.8 1291.2 0.2X +Create a dataset of dates 2209 2270 53 4.5 220.9 0.9X +to_json(date) 9433 9489 90 1.1 943.3 0.2X +write dates to files 6915 6923 8 1.4 691.5 0.3X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 1297 1316 26 7.7 129.7 1.0X -read timestamps from files 16915 17723 963 0.6 1691.5 0.1X -infer timestamps from files 33967 34304 360 0.3 3396.7 0.0X -read date text from files 1095 1100 7 9.1 109.5 1.2X -read date from files 8376 8513 209 1.2 837.6 0.2X -timestamp strings 1807 1816 8 5.5 180.7 0.7X -parse timestamps from Dataset[String] 18189 18242 74 0.5 1818.9 0.1X -infer timestamps from Dataset[String] 37906 38547 571 0.3 3790.6 0.0X -date strings 2191 2194 4 4.6 219.1 0.6X -parse dates from Dataset[String] 11593 11625 33 0.9 1159.3 0.1X -from_json(timestamp) 22589 22650 101 0.4 2258.9 0.1X -from_json(date) 16479 16619 159 0.6 1647.9 0.1X +read timestamp text from files 2395 2412 17 4.2 239.5 1.0X +read timestamps from files 47269 47334 89 0.2 4726.9 0.1X +infer timestamps from files 91806 91851 67 0.1 9180.6 0.0X +read date text from files 2118 2133 13 4.7 211.8 1.1X +read date from files 17267 17340 115 0.6 1726.7 0.1X +timestamp strings 3906 3935 26 2.6 390.6 0.6X +parse timestamps from Dataset[String] 52244 52534 279 0.2 5224.4 0.0X +infer timestamps from Dataset[String] 100488 100714 198 0.1 10048.8 0.0X +date strings 4572 4584 12 2.2 457.2 0.5X +parse dates from Dataset[String] 26749 26768 17 0.4 2674.9 0.1X +from_json(timestamp) 71414 71867 556 0.1 7141.4 0.0X +from_json(date) 45322 45549 250 0.2 4532.2 0.1X From 695cb617d42507eded9c7e50bc7cd5333bbe6f83 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 25 May 2020 15:07:41 +0000 Subject: [PATCH 04/14] [SPARK-31771][SQL] Disable Narrow TextStyle for datetime pattern 'G/M/L/E/u/Q/q' ### What changes were proposed in this pull request? Five continuous pattern characters with 'G/M/L/E/u/Q/q' means Narrow-Text Style while we turn to use `java.time.DateTimeFormatterBuilder` since 3.0.0, which output the leading single letter of the value, e.g. `December` would be `D`. In Spark 2.4 they mean Full-Text Style. In this PR, we explicitly disable Narrow-Text Style for these pattern characters. ### Why are the changes needed? Without this change, there will be a silent data change. ### Does this PR introduce _any_ user-facing change? Yes, queries with datetime operations using datetime patterns, e.g. `G/M/L/E/u` will fail if the pattern length is 5 and other patterns, e,g. 'k', 'm' also can accept a certain number of letters. 1. datetime patterns that are not supported by the new parser but the legacy will get SparkUpgradeException, e.g. "GGGGG", "MMMMM", "LLLLL", "EEEEE", "uuuuu", "aa", "aaa". 2 options are given to end-users, one is to use legacy mode, and the other is to follow the new online doc for correct datetime patterns 2, datetime patterns that are not supported by both the new parser and the legacy, e.g. "QQQQQ", "qqqqq", will get IllegalArgumentException which is captured by Spark internally and results NULL to end-users. ### How was this patch tested? add unit tests Closes #28592 from yaooqinn/SPARK-31771. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- docs/sql-ref-datetime-pattern.md | 49 +- .../sql/catalyst/csv/UnivocityParser.scala | 4 +- .../expressions/datetimeExpressions.scala | 4 + .../sql/catalyst/json/JacksonParser.scala | 4 +- .../sql/catalyst/util/DateFormatter.scala | 16 +- .../util/DateTimeFormatterHelper.scala | 33 + .../catalyst/util/TimestampFormatter.scala | 15 +- .../expressions/DateExpressionsSuite.scala | 2 +- .../util/DateTimeFormatterHelperSuite.scala | 12 +- .../sql/util/TimestampFormatterSuite.scala | 11 + .../sql-tests/inputs/datetime-legacy.sql | 2 + .../resources/sql-tests/inputs/datetime.sql | 20 + .../sql-tests/results/ansi/datetime.sql.out | 163 ++- .../sql-tests/results/datetime-legacy.sql.out | 958 ++++++++++++++++++ .../sql-tests/results/datetime.sql.out | 163 ++- .../native/stringCastAndExpressions.sql.out | 15 +- .../apache/spark/sql/DateFunctionsSuite.scala | 20 +- 17 files changed, 1441 insertions(+), 50 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/datetime-legacy.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out diff --git a/docs/sql-ref-datetime-pattern.md b/docs/sql-ref-datetime-pattern.md index df19b9ce4c082..4275f03335b33 100644 --- a/docs/sql-ref-datetime-pattern.md +++ b/docs/sql-ref-datetime-pattern.md @@ -30,25 +30,25 @@ Spark uses pattern letters in the following table for date and timestamp parsing |Symbol|Meaning|Presentation|Examples| |------|-------|------------|--------| -|**G**|era|text|AD; Anno Domini; A| +|**G**|era|text|AD; Anno Domini| |**y**|year|year|2020; 20| -|**D**|day-of-year|number|189| -|**M/L**|month-of-year|number/text|7; 07; Jul; July; J| -|**d**|day-of-month|number|28| +|**D**|day-of-year|number(3)|189| +|**M/L**|month-of-year|month|7; 07; Jul; July| +|**d**|day-of-month|number(3)|28| |**Q/q**|quarter-of-year|number/text|3; 03; Q3; 3rd quarter| |**Y**|week-based-year|year|1996; 96| -|**w**|week-of-week-based-year|number|27| -|**W**|week-of-month|number|4| -|**E**|day-of-week|text|Tue; Tuesday; T| -|**u**|localized day-of-week|number/text|2; 02; Tue; Tuesday; T| -|**F**|week-of-month|number|3| -|**a**|am-pm-of-day|text|PM| -|**h**|clock-hour-of-am-pm (1-12)|number|12| -|**K**|hour-of-am-pm (0-11)|number|0| -|**k**|clock-hour-of-day (1-24)|number|0| -|**H**|hour-of-day (0-23)|number|0| -|**m**|minute-of-hour|number|30| -|**s**|second-of-minute|number|55| +|**w**|week-of-week-based-year|number(2)|27| +|**W**|week-of-month|number(1)|4| +|**E**|day-of-week|text|Tue; Tuesday| +|**u**|localized day-of-week|number/text|2; 02; Tue; Tuesday| +|**F**|week-of-month|number(1)|3| +|**a**|am-pm-of-day|am-pm|PM| +|**h**|clock-hour-of-am-pm (1-12)|number(2)|12| +|**K**|hour-of-am-pm (0-11)|number(2)|0| +|**k**|clock-hour-of-day (1-24)|number(2)|0| +|**H**|hour-of-day (0-23)|number(2)|0| +|**m**|minute-of-hour|number(2)|30| +|**s**|second-of-minute|number(2)|55| |**S**|fraction-of-second|fraction|978| |**V**|time-zone ID|zone-id|America/Los_Angeles; Z; -08:30| |**z**|time-zone name|zone-name|Pacific Standard Time; PST| @@ -63,9 +63,9 @@ Spark uses pattern letters in the following table for date and timestamp parsing The count of pattern letters determines the format. -- Text: The text style is determined based on the number of pattern letters used. Less than 4 pattern letters will use the short form. Exactly 4 pattern letters will use the full form. Exactly 5 pattern letters will use the narrow form. Six or more letters will fail. +- Text: The text style is determined based on the number of pattern letters used. Less than 4 pattern letters will use the short form. Exactly 4 pattern letters will use the full form. Exactly 5 pattern letters will use the narrow form. 5 or more letters will fail. -- Number: If the count of letters is one, then the value is output using the minimum number of digits and without padding. Otherwise, the count of digits is used as the width of the output field, with the value zero-padded as necessary. The following pattern letters have constraints on the count of letters. Only one letter 'F' can be specified. Up to two letters of 'd', 'H', 'h', 'K', 'k', 'm', and 's' can be specified. Up to three letters of 'D' can be specified. +- Number(n): The n here represents the maximum count of letters this type of datetime pattern can be used. If the count of letters is one, then the value is output using the minimum number of digits and without padding. Otherwise, the count of digits is used as the width of the output field, with the value zero-padded as necessary. - Number/Text: If the count of pattern letters is 3 or greater, use the Text rules above. Otherwise use the Number rules above. @@ -76,7 +76,7 @@ The count of pattern letters determines the format. - Year: The count of letters determines the minimum field width below which padding is used. If the count of letters is two, then a reduced two digit form is used. For printing, this outputs the rightmost two digits. For parsing, this will parse using the base value of 2000, resulting in a year within the range 2000 to 2099 inclusive. If the count of letters is less than four (but not two), then the sign is only output for negative years. Otherwise, the sign is output if the pad width is exceeded when 'G' is not present. -- Month: If the number of pattern letters is 3 or more, the month is interpreted as text; otherwise, it is interpreted as a number. The text form is depend on letters - 'M' denotes the 'standard' form, and 'L' is for 'stand-alone' form. The difference between the 'standard' and 'stand-alone' forms is trickier to describe as there is no difference in English. However, in other languages there is a difference in the word used when the text is used alone, as opposed to in a complete date. For example, the word used for a month when used alone in a date picker is different to the word used for month in association with a day and year in a date. In Russian, 'Июль' is the stand-alone form of July, and 'Июля' is the standard form. Here are examples for all supported pattern letters (more than 5 letters is invalid): +- Month: If the number of pattern letters is 3 or more, the month is interpreted as text; otherwise, it is interpreted as a number. The text form is depend on letters - 'M' denotes the 'standard' form, and 'L' is for 'stand-alone' form. The difference between the 'standard' and 'stand-alone' forms is trickier to describe as there is no difference in English. However, in other languages there is a difference in the word used when the text is used alone, as opposed to in a complete date. For example, the word used for a month when used alone in a date picker is different to the word used for month in association with a day and year in a date. In Russian, 'Июль' is the stand-alone form of July, and 'Июля' is the standard form. Here are examples for all supported pattern letters (more than 4 letters is invalid): - `'M'` or `'L'`: Month number in a year starting from 1. There is no difference between 'M' and 'L'. Month from 1 to 9 are printed without padding. ```sql spark-sql> select date_format(date '1970-01-01', "M"); @@ -119,13 +119,8 @@ The count of pattern letters determines the format. spark-sql> select to_csv(named_struct('date', date '1970-01-01'), map('dateFormat', 'LLLL', 'locale', 'RU')); январь ``` - - `'LLLLL'` or `'MMMMM'`: Narrow textual representation of standard or stand-alone forms. Typically it is a single letter. - ```sql - spark-sql> select date_format(date '1970-07-01', "LLLLL"); - J - spark-sql> select date_format(date '1970-01-01', "MMMMM"); - J - ``` + +- am-pm: This outputs the am-pm-of-day. Pattern letter count must be 1. - Zone ID(V): This outputs the display the time-zone ID. Pattern letter count must be 2. @@ -147,5 +142,3 @@ More details for the text style: - Short Form: Short text, typically an abbreviation. For example, day-of-week Monday might output "Mon". - Full Form: Full text, typically the full description. For example, day-of-week Monday might output "Monday". - -- Narrow Form: Narrow text, typically a single letter. For example, day-of-week Monday might output "M". diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 8e87a82769471..f2bb7db895ca2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -85,13 +85,13 @@ class UnivocityParser( // We preallocate it avoid unnecessary allocations. private val noRows = None - private val timestampFormatter = TimestampFormatter( + private lazy val timestampFormatter = TimestampFormatter( options.timestampFormat, options.zoneId, options.locale, legacyFormat = FAST_DATE_FORMAT, needVarLengthSecondFraction = true) - private val dateFormatter = DateFormatter( + private lazy val dateFormatter = DateFormatter( options.dateFormat, options.zoneId, options.locale, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index afc57aa546fe8..7dc008a2e5df8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -880,6 +880,7 @@ abstract class ToTimestamp legacyFormat = SIMPLE_DATE_FORMAT, needVarLengthSecondFraction = true) } catch { + case e: SparkUpgradeException => throw e case NonFatal(_) => null } @@ -1061,6 +1062,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ legacyFormat = SIMPLE_DATE_FORMAT, needVarLengthSecondFraction = false) } catch { + case e: SparkUpgradeException => throw e case NonFatal(_) => null } @@ -1076,6 +1078,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ try { UTF8String.fromString(formatter.format(time.asInstanceOf[Long] * MICROS_PER_SECOND)) } catch { + case e: SparkUpgradeException => throw e case NonFatal(_) => null } } @@ -1093,6 +1096,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ needVarLengthSecondFraction = false) .format(time.asInstanceOf[Long] * MICROS_PER_SECOND)) } catch { + case e: SparkUpgradeException => throw e case NonFatal(_) => null } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index ef987931e928a..c4f6121723491 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -56,13 +56,13 @@ class JacksonParser( private val factory = options.buildJsonFactory() - private val timestampFormatter = TimestampFormatter( + private lazy val timestampFormatter = TimestampFormatter( options.timestampFormat, options.zoneId, options.locale, legacyFormat = FAST_DATE_FORMAT, needVarLengthSecondFraction = true) - private val dateFormatter = DateFormatter( + private lazy val dateFormatter = DateFormatter( options.dateFormat, options.zoneId, options.locale, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala index 8261f57916fa2..06e1cdc27e7d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.text.SimpleDateFormat import java.time.{LocalDate, ZoneId} +import java.time.format.DateTimeFormatter import java.util.{Date, Locale} import org.apache.commons.lang3.time.FastDateFormat @@ -33,6 +34,8 @@ sealed trait DateFormatter extends Serializable { def format(days: Int): String def format(date: Date): String def format(localDate: LocalDate): String + + def validatePatternString(): Unit } class Iso8601DateFormatter( @@ -70,6 +73,12 @@ class Iso8601DateFormatter( override def format(date: Date): String = { legacyFormatter.format(date) } + + override def validatePatternString(): Unit = { + try { + formatter + } catch checkLegacyFormatter(pattern, legacyFormatter.validatePatternString) + } } trait LegacyDateFormatter extends DateFormatter { @@ -93,6 +102,7 @@ class LegacyFastDateFormatter(pattern: String, locale: Locale) extends LegacyDat private lazy val fdf = FastDateFormat.getInstance(pattern, locale) override def parseToDate(s: String): Date = fdf.parse(s) override def format(d: Date): String = fdf.format(d) + override def validatePatternString(): Unit = fdf } class LegacySimpleDateFormatter(pattern: String, locale: Locale) extends LegacyDateFormatter { @@ -100,6 +110,8 @@ class LegacySimpleDateFormatter(pattern: String, locale: Locale) extends LegacyD private lazy val sdf = new SimpleDateFormat(pattern, locale) override def parseToDate(s: String): Date = sdf.parse(s) override def format(d: Date): String = sdf.format(d) + override def validatePatternString(): Unit = sdf + } object DateFormatter { @@ -118,7 +130,9 @@ object DateFormatter { if (SQLConf.get.legacyTimeParserPolicy == LEGACY) { getLegacyFormatter(pattern, zoneId, locale, legacyFormat) } else { - new Iso8601DateFormatter(pattern, zoneId, locale, legacyFormat) + val df = new Iso8601DateFormatter(pattern, zoneId, locale, legacyFormat) + df.validatePatternString() + df } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala index 35f95dbffca6e..0ea54c28cb285 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala @@ -117,6 +117,34 @@ trait DateTimeFormatterHelper { s"set ${SQLConf.LEGACY_TIME_PARSER_POLICY.key} to LEGACY to restore the behavior " + s"before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.", e) } + + /** + * When the new DateTimeFormatter failed to initialize because of invalid datetime pattern, it + * will throw IllegalArgumentException. If the pattern can be recognized by the legacy formatter + * it will raise SparkUpgradeException to tell users to restore the previous behavior via LEGACY + * policy or follow our guide to correct their pattern. Otherwise, the original + * IllegalArgumentException will be thrown. + * + * @param pattern the date time pattern + * @param tryLegacyFormatter a func to capture exception, identically which forces a legacy + * datetime formatter to be initialized + */ + + protected def checkLegacyFormatter( + pattern: String, + tryLegacyFormatter: => Unit): PartialFunction[Throwable, DateTimeFormatter] = { + case e: IllegalArgumentException => + try { + tryLegacyFormatter + } catch { + case _: Throwable => throw e + } + throw new SparkUpgradeException("3.0", s"Fail to recognize '$pattern' pattern in the" + + s" DateTimeFormatter. 1) You can set ${SQLConf.LEGACY_TIME_PARSER_POLICY.key} to LEGACY" + + s" to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern" + + s" with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html", + e) + } } private object DateTimeFormatterHelper { @@ -190,6 +218,8 @@ private object DateTimeFormatterHelper { } final val unsupportedLetters = Set('A', 'c', 'e', 'n', 'N', 'p') + final val unsupportedNarrowTextStyle = + Set("GGGGG", "MMMMM", "LLLLL", "EEEEE", "uuuuu", "QQQQQ", "qqqqq", "uuuuu") /** * In Spark 3.0, we switch to the Proleptic Gregorian calendar and use DateTimeFormatter for @@ -211,6 +241,9 @@ private object DateTimeFormatterHelper { for (c <- patternPart if unsupportedLetters.contains(c)) { throw new IllegalArgumentException(s"Illegal pattern character: $c") } + for (style <- unsupportedNarrowTextStyle if patternPart.contains(style)) { + throw new IllegalArgumentException(s"Too many pattern letters: ${style.head}") + } // The meaning of 'u' was day number of week in SimpleDateFormat, it was changed to year // in DateTimeFormatter. Substitute 'u' to 'e' and use DateTimeFormatter to parse the // string. If parsable, return the result; otherwise, fall back to 'u', and then use the diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index 1a6e5e4400ffb..de2fd312b7db5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -54,6 +54,7 @@ sealed trait TimestampFormatter extends Serializable { def format(us: Long): String def format(ts: Timestamp): String def format(instant: Instant): String + def validatePatternString(): Unit } class Iso8601TimestampFormatter( @@ -99,6 +100,12 @@ class Iso8601TimestampFormatter( override def format(ts: Timestamp): String = { legacyFormatter.format(ts) } + + override def validatePatternString(): Unit = { + try { + formatter + } catch checkLegacyFormatter(pattern, legacyFormatter.validatePatternString) + } } /** @@ -202,6 +209,8 @@ class LegacyFastTimestampFormatter( override def format(instant: Instant): String = { format(instantToMicros(instant)) } + + override def validatePatternString(): Unit = fastDateFormat } class LegacySimpleTimestampFormatter( @@ -231,6 +240,8 @@ class LegacySimpleTimestampFormatter( override def format(instant: Instant): String = { format(instantToMicros(instant)) } + + override def validatePatternString(): Unit = sdf } object LegacyDateFormats extends Enumeration { @@ -255,8 +266,10 @@ object TimestampFormatter { if (SQLConf.get.legacyTimeParserPolicy == LEGACY) { getLegacyFormatter(pattern, zoneId, locale, legacyFormat) } else { - new Iso8601TimestampFormatter( + val tf = new Iso8601TimestampFormatter( pattern, zoneId, locale, legacyFormat, needVarLengthSecondFraction) + tf.validatePatternString() + tf } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 87062f2d4ef38..02d6d847dc063 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -267,7 +267,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // Test escaping of format GenerateUnsafeProjection.generate( - DateFormatClass(Literal(ts), Literal("\"quote"), JST_OPT) :: Nil) + DateFormatClass(Literal(ts), Literal("\""), JST_OPT) :: Nil) // SPARK-28072 The codegen path should work checkEvaluation( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelperSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelperSuite.scala index 817e503584324..caf7bdde10122 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelperSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelperSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.util -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkFunSuite, SparkUpgradeException} import org.apache.spark.sql.catalyst.util.DateTimeFormatterHelper._ class DateTimeFormatterHelperSuite extends SparkFunSuite { @@ -40,6 +40,16 @@ class DateTimeFormatterHelperSuite extends SparkFunSuite { val e = intercept[IllegalArgumentException](convertIncompatiblePattern(s"yyyy-MM-dd $l G")) assert(e.getMessage === s"Illegal pattern character: $l") } + unsupportedNarrowTextStyle.foreach { style => + val e1 = intercept[IllegalArgumentException] { + convertIncompatiblePattern(s"yyyy-MM-dd $style") + } + assert(e1.getMessage === s"Too many pattern letters: ${style.head}") + val e2 = intercept[IllegalArgumentException] { + convertIncompatiblePattern(s"yyyy-MM-dd $style${style.head}") + } + assert(e2.getMessage === s"Too many pattern letters: ${style.head}") + } assert(convertIncompatiblePattern("yyyy-MM-dd uuuu") === "uuuu-MM-dd eeee") assert(convertIncompatiblePattern("yyyy-MM-dd EEEE") === "uuuu-MM-dd EEEE") assert(convertIncompatiblePattern("yyyy-MM-dd'e'HH:mm:ss") === "uuuu-MM-dd'e'HH:mm:ss") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala index dccb3defe3728..4324d3cff63d7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -396,4 +396,15 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers val micros = formatter.parse("2009 11") assert(micros === date(2009, 1, 1, 11)) } + + test("explicitly forbidden datetime patterns") { + // not support by the legacy one too + Seq("QQQQQ", "qqqqq", "A", "c", "e", "n", "N", "p").foreach { pattern => + intercept[IllegalArgumentException](TimestampFormatter(pattern, UTC).format(0)) + } + // supported by the legacy one, then we will suggest users with SparkUpgradeException + Seq("GGGGG", "MMMMM", "LLLLL", "EEEEE", "uuuuu", "aa", "aaa").foreach { pattern => + intercept[SparkUpgradeException](TimestampFormatter(pattern, UTC).format(0)) + } + } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-legacy.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-legacy.sql new file mode 100644 index 0000000000000..daec2b40a620b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime-legacy.sql @@ -0,0 +1,2 @@ +--SET spark.sql.legacy.timeParserPolicy=LEGACY +--IMPORT datetime.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql index 0fb373f419e7e..663c62f1a6f66 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -140,3 +140,23 @@ select to_date("16", "dd"); select to_date("02-29", "MM-dd"); select to_timestamp("2019 40", "yyyy mm"); select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss"); + +-- Unsupported narrow text style +select date_format(date '2020-05-23', 'GGGGG'); +select date_format(date '2020-05-23', 'MMMMM'); +select date_format(date '2020-05-23', 'LLLLL'); +select date_format(timestamp '2020-05-23', 'EEEEE'); +select date_format(timestamp '2020-05-23', 'uuuuu'); +select date_format('2020-05-23', 'QQQQQ'); +select date_format('2020-05-23', 'qqqqq'); +select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG'); +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE'); +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE'); +select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE'); +select from_unixtime(12345, 'MMMMM'); +select from_unixtime(54321, 'QQQQQ'); +select from_unixtime(23456, 'aaaaa'); +select from_json('{"time":"26/October/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')); +select from_json('{"date":"26/October/2015"}', 'date Date', map('dateFormat', 'dd/MMMMM/yyyy')); +select from_csv('26/October/2015', 'time Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')); +select from_csv('26/October/2015', 'date Date', map('dateFormat', 'dd/MMMMM/yyyy')); diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index 2e61cb8cb8c3f..5857a0ac90c70 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 92 +-- Number of queries: 116 -- !query @@ -838,3 +838,164 @@ select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss") struct -- !query output 2019-01-01 10:10:10 + + +-- !query +select date_format(date '2020-05-23', 'GGGGG') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'GGGGG' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select date_format(date '2020-05-23', 'MMMMM') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'MMMMM' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select date_format(date '2020-05-23', 'LLLLL') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'LLLLL' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select date_format(timestamp '2020-05-23', 'EEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select date_format(timestamp '2020-05-23', 'uuuuu') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'uuuuu' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select date_format('2020-05-23', 'QQQQQ') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Too many pattern letters: Q + + +-- !query +select date_format('2020-05-23', 'qqqqq') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Too many pattern letters: q + + +-- !query +select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'yyyy-MM-dd GGGGG' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd MM yyyy EEEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select from_unixtime(12345, 'MMMMM') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'MMMMM' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select from_unixtime(54321, 'QQQQQ') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(23456, 'aaaaa') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'aaaaa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select from_json('{"time":"26/October/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select from_json('{"date":"26/October/2015"}', 'date Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select from_csv('26/October/2015', 'time Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select from_csv('26/October/2015', 'date Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out new file mode 100644 index 0000000000000..8a726efafad89 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -0,0 +1,958 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 116 + + +-- !query +select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00 1931-01-07 00:30:00 NULL + + +-- !query +select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL + + +-- !query +select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 NULL + + +-- !query +select TIMESTAMP_SECONDS(1230219000123123) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_SECONDS(-1230219000123123) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_MILLIS(92233720368547758) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_MILLIS(-92233720368547758) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select current_date = current_date(), current_timestamp = current_timestamp() +-- !query schema +struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean> +-- !query output +true true + + +-- !query +select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 2016-12-31 + + +-- !query +select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 + + +-- !query +select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), dayofweek('1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +7 5 7 NULL 6 + + +-- !query +create temporary view ttf1 as select * from values + (1, 2), + (2, 3) + as ttf1(current_date, current_timestamp) +-- !query schema +struct<> +-- !query output + + + +-- !query +select current_date, current_timestamp from ttf1 +-- !query schema +struct +-- !query output +1 2 +2 3 + + +-- !query +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2 +-- !query schema +struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean,a:int,b:int> +-- !query output +true true 1 2 +true true 2 3 + + +-- !query +select a, b from ttf2 order by a, current_date +-- !query schema +struct +-- !query output +1 2 +2 3 + + +-- !query +select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), weekday('1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +5 3 5 NULL 4 + + +-- !query +select year('1500-01-01'), month('1500-01-01'), dayOfYear('1500-01-01') +-- !query schema +struct +-- !query output +1500 1 1 + + +-- !query +select date '2019-01-01\t' +-- !query schema +struct +-- !query output +2019-01-01 + + +-- !query +select timestamp '2019-01-01\t' +-- !query schema +struct +-- !query output +2019-01-01 00:00:00 + + +-- !query +select timestamp'2011-11-11 11:11:11' + interval '2' day +-- !query schema +struct +-- !query output +2011-11-13 11:11:11 + + +-- !query +select timestamp'2011-11-11 11:11:11' - interval '2' day +-- !query schema +struct +-- !query output +2011-11-09 11:11:11 + + +-- !query +select date'2011-11-11 11:11:11' + interval '2' second +-- !query schema +struct +-- !query output +2011-11-11 + + +-- !query +select date'2011-11-11 11:11:11' - interval '2' second +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select '2011-11-11' - interval '2' day +-- !query schema +struct +-- !query output +2011-11-09 00:00:00 + + +-- !query +select '2011-11-11 11:11:11' - interval '2' second +-- !query schema +struct +-- !query output +2011-11-11 11:11:09 + + +-- !query +select '1' - interval '2' second +-- !query schema +struct +-- !query output +NULL + + +-- !query +select 1 - interval '2' second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve '1 + (- INTERVAL '2 seconds')' due to data type mismatch: argument 1 requires timestamp type, however, '1' is of int type.; line 1 pos 7 + + +-- !query +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +-- !query schema +struct +-- !query output +2078 hours 48 minutes 47.654322 seconds + + +-- !query +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +-- !query schema +struct +-- !query output +-2078 hours -48 minutes -47.654322 seconds + + +-- !query +select timestamp'2019-10-06 10:11:12.345678' - null +-- !query schema +struct +-- !query output +NULL + + +-- !query +select null - timestamp'2019-10-06 10:11:12.345678' +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_add('2011-11-11', 1Y) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1S) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1L) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 1L)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1L' is of bigint type.; line 1 pos 7 + + +-- !query +select date_add('2011-11-11', 1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 1.0BD)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1.0BD' is of decimal(2,1) type.; line 1 pos 7 + + +-- !query +select date_add('2011-11-11', 1E1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7 + + +-- !query +select date_add('2011-11-11', '1') +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', '1.2') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +The second argument of 'date_add' function needs to be an integer.; + + +-- !query +select date_add(date'2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add(timestamp'2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_sub(date'2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub(date'2011-11-11', '1') +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub(date'2011-11-11', '1.2') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +The second argument of 'date_sub' function needs to be an integer.; + + +-- !query +select date_sub(timestamp'2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub(null, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_sub(date'2011-11-11', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date'2011-11-11' + 1E1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(DATE '2011-11-11', 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7 + + +-- !query +select date'2011-11-11' + '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(DATE '2011-11-11', CAST('1' AS DOUBLE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST('1' AS DOUBLE)' is of double type.; line 1 pos 7 + + +-- !query +select null + date '2001-09-28' +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date '2001-09-28' + 7Y +-- !query schema +struct +-- !query output +2001-10-05 + + +-- !query +select 7S + date '2001-09-28' +-- !query schema +struct +-- !query output +2001-10-05 + + +-- !query +select date '2001-10-01' - 7 +-- !query schema +struct +-- !query output +2001-09-24 + + +-- !query +select date '2001-10-01' - '7' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_sub(DATE '2001-10-01', CAST('7' AS DOUBLE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST('7' AS DOUBLE)' is of double type.; line 1 pos 7 + + +-- !query +select date '2001-09-28' + null +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date '2001-09-28' - null +-- !query schema +struct +-- !query output +NULL + + +-- !query +create temp view v as select '1' str +-- !query schema +struct<> +-- !query output + + + +-- !query +select date_add('2011-11-11', str) from v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(CAST('2011-11-11' AS DATE), v.`str`)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'v.`str`' is of string type.; line 1 pos 7 + + +-- !query +select date_sub('2011-11-11', str) from v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_sub(CAST('2011-11-11' AS DATE), v.`str`)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'v.`str`' is of string type.; line 1 pos 7 + + +-- !query +select null - date '2019-10-06' +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date '2001-10-01' - date '2001-09-28' +-- !query schema +struct +-- !query output +3 days + + +-- !query +select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select date_format(timestamp '2019-10-06', 'yyyy-MM-dd uuee') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'e' + + +-- !query +select date_format(timestamp '2019-10-06', 'yyyy-MM-dd uucc') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'c' + + +-- !query +select date_format(timestamp '2019-10-06', 'yyyy-MM-dd uuuu') +-- !query schema +struct +-- !query output +2019-10-06 0007 + + +-- !query +select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.012 + + +-- !query +select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("16", "dd") +-- !query schema +struct +-- !query output +1970-01-16 00:00:00 + + +-- !query +select to_timestamp("02-29", "MM-dd") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date("16", "dd") +-- !query schema +struct +-- !query output +1970-01-16 + + +-- !query +select to_date("02-29", "MM-dd") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("2019 40", "yyyy mm") +-- !query schema +struct +-- !query output +2019-01-01 00:40:00 + + +-- !query +select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss") +-- !query schema +struct +-- !query output +2019-01-01 10:10:10 + + +-- !query +select date_format(date '2020-05-23', 'GGGGG') +-- !query schema +struct +-- !query output +AD + + +-- !query +select date_format(date '2020-05-23', 'MMMMM') +-- !query schema +struct +-- !query output +May + + +-- !query +select date_format(date '2020-05-23', 'LLLLL') +-- !query schema +struct +-- !query output +May + + +-- !query +select date_format(timestamp '2020-05-23', 'EEEEE') +-- !query schema +struct +-- !query output +Saturday + + +-- !query +select date_format(timestamp '2020-05-23', 'uuuuu') +-- !query schema +struct +-- !query output +00006 + + +-- !query +select date_format('2020-05-23', 'QQQQQ') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'Q' + + +-- !query +select date_format('2020-05-23', 'qqqqq') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'q' + + +-- !query +select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') +-- !query schema +struct +-- !query output +2020-05-22 00:00:00 + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct +-- !query output +2020-05-22 00:00:00 + + +-- !query +select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct +-- !query output +1590130800 + + +-- !query +select from_unixtime(12345, 'MMMMM') +-- !query schema +struct +-- !query output +December + + +-- !query +select from_unixtime(54321, 'QQQQQ') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(23456, 'aaaaa') +-- !query schema +struct +-- !query output +PM + + +-- !query +select from_json('{"time":"26/October/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"time":2015-10-26 00:00:00} + + +-- !query +select from_json('{"date":"26/October/2015"}', 'date Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"date":2015-10-26} + + +-- !query +select from_csv('26/October/2015', 'time Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"time":2015-10-26 00:00:00} + + +-- !query +select from_csv('26/October/2015', 'date Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"date":2015-10-26} diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index 4b879fcfbfc5b..7cacaec42c813 100755 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 91 +-- Number of queries: 116 -- !query @@ -810,3 +810,164 @@ select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss") struct -- !query output 2019-01-01 10:10:10 + + +-- !query +select date_format(date '2020-05-23', 'GGGGG') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'GGGGG' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select date_format(date '2020-05-23', 'MMMMM') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'MMMMM' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select date_format(date '2020-05-23', 'LLLLL') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'LLLLL' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select date_format(timestamp '2020-05-23', 'EEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select date_format(timestamp '2020-05-23', 'uuuuu') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'uuuuu' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select date_format('2020-05-23', 'QQQQQ') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Too many pattern letters: Q + + +-- !query +select date_format('2020-05-23', 'qqqqq') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Too many pattern letters: q + + +-- !query +select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'yyyy-MM-dd GGGGG' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd MM yyyy EEEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select from_unixtime(12345, 'MMMMM') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'MMMMM' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select from_unixtime(54321, 'QQQQQ') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(23456, 'aaaaa') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'aaaaa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select from_json('{"time":"26/October/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select from_json('{"date":"26/October/2015"}', 'date Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select from_csv('26/October/2015', 'time Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select from_csv('26/October/2015', 'date Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out index 8353c7e73d0bb..02944c268ed21 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out @@ -136,9 +136,10 @@ NULL -- !query select to_timestamp('2018-01-01', a) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'aa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -152,9 +153,10 @@ NULL -- !query select to_unix_timestamp('2018-01-01', a) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'aa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -168,9 +170,10 @@ NULL -- !query select unix_timestamp('2018-01-01', a) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'aa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 14e6ee2b04c14..c12468a4e70f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -23,7 +23,7 @@ import java.time.{Instant, LocalDateTime, ZoneId} import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit -import org.apache.spark.SparkException +import org.apache.spark.{SparkException, SparkUpgradeException} import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{CEST, LA} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.functions._ @@ -450,9 +450,9 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer( df.select(to_date(col("s"), "yyyy-hh-MM")), Seq(Row(null), Row(null), Row(null))) - checkAnswer( - df.select(to_date(col("s"), "yyyy-dd-aa")), - Seq(Row(null), Row(null), Row(null))) + val e = intercept[SparkUpgradeException](df.select(to_date(col("s"), "yyyy-dd-aa")).collect()) + assert(e.getCause.isInstanceOf[IllegalArgumentException]) + assert(e.getMessage.contains("You may get a different result due to the upgrading of Spark")) // february val x1 = "2016-02-29" @@ -618,8 +618,16 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { Row(secs(ts4.getTime)), Row(null), Row(secs(ts3.getTime)), Row(null))) // invalid format - checkAnswer(df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd aa:HH:ss')"), Seq( - Row(null), Row(null), Row(null), Row(null))) + val invalid = df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd aa:HH:ss')") + if (legacyParserPolicy == "legacy") { + checkAnswer(invalid, + Seq(Row(null), Row(null), Row(null), Row(null))) + } else { + val e = intercept[SparkUpgradeException](invalid.collect()) + assert(e.getCause.isInstanceOf[IllegalArgumentException]) + assert( + e.getMessage.contains("You may get a different result due to the upgrading of Spark")) + } // february val y1 = "2016-02-29" From 6c80ebbccb7f354f645dd63a73114332d26f901f Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 25 May 2020 18:36:02 -0700 Subject: [PATCH 05/14] [SPARK-31818][SQL] Fix pushing down filters with `java.time.Instant` values in ORC ### What changes were proposed in this pull request? Convert `java.time.Instant` to `java.sql.Timestamp` in pushed down filters to ORC datasource when Java 8 time API enabled. ### Why are the changes needed? The changes fix the exception raised while pushing date filters when `spark.sql.datetime.java8API.enabled` is set to `true`: ``` java.lang.IllegalArgumentException: Wrong value class java.time.Instant for TIMESTAMP.EQUALS leaf at org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl$PredicateLeafImpl.checkLiteralType(SearchArgumentImpl.java:192) at org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl$PredicateLeafImpl.(SearchArgumentImpl.java:75) ``` ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? Added tests to `OrcFilterSuite`. Closes #28636 from MaxGekk/orc-timestamp-filter-pushdown. Authored-by: Max Gekk Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/dsl/package.scala | 3 +- .../datasources/orc/OrcFilters.scala | 6 +- .../datasources/orc/OrcFilterSuite.scala | 56 +++++++++++-------- .../datasources/orc/OrcFilters.scala | 6 +- .../datasources/orc/OrcFilterSuite.scala | 54 +++++++++++------- 5 files changed, 77 insertions(+), 48 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index fe3fea5e35b1b..26f5bee72092c 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst import java.sql.{Date, Timestamp} -import java.time.LocalDate +import java.time.{Instant, LocalDate} import scala.language.implicitConversions @@ -152,6 +152,7 @@ package object dsl { implicit def bigDecimalToLiteral(d: java.math.BigDecimal): Literal = Literal(d) implicit def decimalToLiteral(d: Decimal): Literal = Literal(d) implicit def timestampToLiteral(t: Timestamp): Literal = Literal(t) + implicit def instantToLiteral(i: Instant): Literal = Literal(i) implicit def binaryToLiteral(a: Array[Byte]): Literal = Literal(a) implicit def symbolToUnresolvedAttribute(s: Symbol): analysis.UnresolvedAttribute = diff --git a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index a01d5a44da714..b68563956c82c 100644 --- a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.orc -import java.time.LocalDate +import java.time.{Instant, LocalDate} import org.apache.orc.storage.common.`type`.HiveDecimal import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} @@ -26,7 +26,7 @@ import org.apache.orc.storage.ql.io.sarg.SearchArgumentFactory.newBuilder import org.apache.orc.storage.serde2.io.HiveDecimalWritable import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.util.DateTimeUtils.{localDateToDays, toJavaDate} +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{instantToMicros, localDateToDays, toJavaDate, toJavaTimestamp} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.quoteIfNeeded import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ @@ -167,6 +167,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { new HiveDecimalWritable(HiveDecimal.create(value.asInstanceOf[java.math.BigDecimal])) case _: DateType if value.isInstanceOf[LocalDate] => toJavaDate(localDateToDays(value.asInstanceOf[LocalDate])) + case _: TimestampType if value.isInstanceOf[Instant] => + toJavaTimestamp(instantToMicros(value.asInstanceOf[Instant])) case _ => value } diff --git a/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index a1c325e7bb876..88b4b243b543a 100644 --- a/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -245,29 +245,41 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { } test("filter pushdown - timestamp") { - val timeString = "2015-08-20 14:57:00" - val timestamps = (1 to 4).map { i => - val milliseconds = Timestamp.valueOf(timeString).getTime + i * 3600 - new Timestamp(milliseconds) - } - withOrcDataFrame(timestamps.map(Tuple1(_))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + val input = Seq( + "1000-01-01 01:02:03", + "1582-10-01 00:11:22", + "1900-01-01 23:59:59", + "2020-05-25 10:11:12").map(Timestamp.valueOf) - checkFilterPredicate($"_1" === timestamps(0), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < timestamps(1), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(timestamps(0)) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(timestamps(0)) <=> $"_1", - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(timestamps(1)) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(timestamps(2)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(0)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(3)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + withOrcFile(input.map(Tuple1(_))) { path => + Seq(false, true).foreach { java8Api => + withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString) { + readFile(path) { implicit df => + val timestamps = input.map(Literal(_)) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === timestamps(0), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < timestamps(1), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(timestamps(0)) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate( + Literal(timestamps(0)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(timestamps(1)) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate( + Literal(timestamps(2)) < $"_1", + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate( + Literal(timestamps(0)) >= $"_1", + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(3)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + } + } + } } } diff --git a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 445a52cece1c3..4b642080d25ad 100644 --- a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.orc -import java.time.LocalDate +import java.time.{Instant, LocalDate} import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.ql.io.sarg.{PredicateLeaf, SearchArgument} @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory.newBuilder import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.util.DateTimeUtils.{localDateToDays, toJavaDate} +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{instantToMicros, localDateToDays, toJavaDate, toJavaTimestamp} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.quoteIfNeeded import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ @@ -167,6 +167,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { new HiveDecimalWritable(HiveDecimal.create(value.asInstanceOf[java.math.BigDecimal])) case _: DateType if value.isInstanceOf[LocalDate] => toJavaDate(localDateToDays(value.asInstanceOf[LocalDate])) + case _: TimestampType if value.isInstanceOf[Instant] => + toJavaTimestamp(instantToMicros(value.asInstanceOf[Instant])) case _ => value } diff --git a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index 815af05beb002..2263179515a5f 100644 --- a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -246,29 +246,41 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { } test("filter pushdown - timestamp") { - val timeString = "2015-08-20 14:57:00" - val timestamps = (1 to 4).map { i => - val milliseconds = Timestamp.valueOf(timeString).getTime + i * 3600 - new Timestamp(milliseconds) - } - withOrcDataFrame(timestamps.map(Tuple1(_))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === timestamps(0), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + val input = Seq( + "1000-01-01 01:02:03", + "1582-10-01 00:11:22", + "1900-01-01 23:59:59", + "2020-05-25 10:11:12").map(Timestamp.valueOf) - checkFilterPredicate($"_1" < timestamps(1), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) + withOrcFile(input.map(Tuple1(_))) { path => + Seq(false, true).foreach { java8Api => + withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString) { + readFile(path) { implicit df => + val timestamps = input.map(Literal(_)) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - checkFilterPredicate(Literal(timestamps(0)) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate( - Literal(timestamps(0)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(timestamps(1)) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(timestamps(2)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(0)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(3)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" === timestamps(0), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < timestamps(1), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(timestamps(0)) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate( + Literal(timestamps(0)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(timestamps(1)) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate( + Literal(timestamps(2)) < $"_1", + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate( + Literal(timestamps(0)) >= $"_1", + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(3)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + } + } + } } } From df2a1fe131476aac128d63df9b06ec4bca0c2c07 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 25 May 2020 20:36:00 -0700 Subject: [PATCH 06/14] [SPARK-31808][SQL] Makes struct function's output name and class name pretty ### What changes were proposed in this pull request? This PR proposes to set the alias, and class name in its `ExpressionInfo` for `struct`. - Class name in `ExpressionInfo` - from: `org.apache.spark.sql.catalyst.expressions.NamedStruct` - to:`org.apache.spark.sql.catalyst.expressions.CreateNamedStruct` - Alias name: `named_struct(col1, v, ...)` -> `struct(v, ...)` This PR takes over https://github.com/apache/spark/pull/28631 ### Why are the changes needed? To show the correct output name and class names to users. ### Does this PR introduce _any_ user-facing change? Yes. **Before:** ```scala scala> sql("DESC FUNCTION struct").show(false) +------------------------------------------------------------------------------------+ |function_desc | +------------------------------------------------------------------------------------+ |Function: struct | |Class: org.apache.spark.sql.catalyst.expressions.NamedStruct | |Usage: struct(col1, col2, col3, ...) - Creates a struct with the given field values.| +------------------------------------------------------------------------------------+ ``` ```scala scala> sql("SELECT struct(1, 2)").show(false) +------------------------------+ |named_struct(col1, 1, col2, 2)| +------------------------------+ |[1, 2] | +------------------------------+ ``` **After:** ```scala scala> sql("DESC FUNCTION struct").show(false) +------------------------------------------------------------------------------------+ |function_desc | +------------------------------------------------------------------------------------+ |Function: struct | |Class: org.apache.spark.sql.catalyst.expressions.CreateNamedStruct | |Usage: struct(col1, col2, col3, ...) - Creates a struct with the given field values.| +------------------------------------------------------------------------------------+ ``` ```scala scala> sql("SELECT struct(1, 2)").show(false) +------------+ |struct(1, 2)| +------------+ |[1, 2] | +------------+ ``` ### How was this patch tested? Manually tested, and Jenkins tests. Closes #28633 from HyukjinKwon/SPARK-31808. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- .../expressions/complexTypeCreator.scala | 34 ++++++++++++++++--- .../sql/catalyst/parser/AstBuilder.scala | 2 +- .../org/apache/spark/sql/functions.scala | 2 +- .../sql-functions/sql-expression-schema.md | 6 ++-- .../sql-tests/results/group-by-filter.sql.out | 2 +- .../sql-tests/results/group-by.sql.out | 2 +- .../sql-tests/results/struct.sql.out | 2 +- .../typeCoercion/native/mapZipWith.sql.out | 4 +-- .../results/udf/udf-group-by.sql.out | 2 +- 9 files changed, 40 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index 858c91a4d8e86..5212ef3930bc9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion} -import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.{FUNC_ALIAS, FunctionBuilder} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util._ @@ -311,7 +311,12 @@ case object NamePlaceholder extends LeafExpression with Unevaluable { /** * Returns a Row containing the evaluation of all children expressions. */ -object CreateStruct extends FunctionBuilder { +object CreateStruct { + /** + * Returns a named struct with generated names or using the names when available. + * It should not be used for `struct` expressions or functions explicitly called + * by users. + */ def apply(children: Seq[Expression]): CreateNamedStruct = { CreateNamedStruct(children.zipWithIndex.flatMap { case (e: NamedExpression, _) if e.resolved => Seq(Literal(e.name), e) @@ -320,12 +325,23 @@ object CreateStruct extends FunctionBuilder { }) } + /** + * Returns a named struct with a pretty SQL name. It will show the pretty SQL string + * in its output column name as if `struct(...)` was called. Should be + * used for `struct` expressions or functions explicitly called by users. + */ + def create(children: Seq[Expression]): CreateNamedStruct = { + val expr = CreateStruct(children) + expr.setTagValue(FUNC_ALIAS, "struct") + expr + } + /** * Entry to use in the function registry. */ val registryEntry: (String, (ExpressionInfo, FunctionBuilder)) = { val info: ExpressionInfo = new ExpressionInfo( - "org.apache.spark.sql.catalyst.expressions.NamedStruct", + classOf[CreateNamedStruct].getCanonicalName, null, "struct", "_FUNC_(col1, col2, col3, ...) - Creates a struct with the given field values.", @@ -335,7 +351,7 @@ object CreateStruct extends FunctionBuilder { "", "", "") - ("struct", (info, this)) + ("struct", (info, this.create)) } } @@ -433,7 +449,15 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression { """.stripMargin, isNull = FalseLiteral) } - override def prettyName: String = "named_struct" + // There is an alias set at `CreateStruct.create`. If there is an alias, + // this is the struct function explicitly called by a user and we should + // respect it in the SQL string as `struct(...)`. + override def prettyName: String = getTagValue(FUNC_ALIAS).getOrElse("named_struct") + + override def sql: String = getTagValue(FUNC_ALIAS).map { alias => + val childrenSQL = children.indices.filter(_ % 2 == 1).map(children(_).sql).mkString(", ") + s"$alias($childrenSQL)" + }.getOrElse(super.sql) } /** 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 c0cecf8536c39..03571a740df3e 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 @@ -1534,7 +1534,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Create a [[CreateStruct]] expression. */ override def visitStruct(ctx: StructContext): Expression = withOrigin(ctx) { - CreateStruct(ctx.argument.asScala.map(expression)) + CreateStruct.create(ctx.argument.asScala.map(expression)) } /** 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 5481337bf6cee..0cca3e7b47c56 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 @@ -1306,7 +1306,7 @@ object functions { * @since 1.4.0 */ @scala.annotation.varargs - def struct(cols: Column*): Column = withExpr { CreateStruct(cols.map(_.expr)) } + def struct(cols: Column*): Column = withExpr { CreateStruct.create(cols.map(_.expr)) } /** * Creates a new struct column that composes multiple input columns. diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index 23173c8ba1f11..8949b62f0a512 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -2,7 +2,7 @@ ## Summary - Number of queries: 337 - Number of expressions that missing example: 34 - - Expressions missing examples: and,string,tinyint,double,smallint,date,decimal,boolean,float,binary,bigint,int,timestamp,cume_dist,dense_rank,input_file_block_length,input_file_block_start,input_file_name,lag,lead,monotonically_increasing_id,ntile,struct,!,not,or,percent_rank,rank,row_number,spark_partition_id,version,window,positive,count_min_sketch + - Expressions missing examples: and,string,tinyint,double,smallint,date,decimal,boolean,float,binary,bigint,int,timestamp,struct,cume_dist,dense_rank,input_file_block_length,input_file_block_start,input_file_name,lag,lead,monotonically_increasing_id,ntile,!,not,or,percent_rank,rank,row_number,spark_partition_id,version,window,positive,count_min_sketch ## Schema of Built-in Functions | Class name | Function name or alias | Query example | Output schema | | ---------- | ---------------------- | ------------- | ------------- | @@ -79,6 +79,7 @@ | org.apache.spark.sql.catalyst.expressions.CreateArray | array | SELECT array(1, 2, 3) | struct> | | org.apache.spark.sql.catalyst.expressions.CreateMap | map | SELECT map(1.0, '2', 3.0, '4') | struct> | | org.apache.spark.sql.catalyst.expressions.CreateNamedStruct | named_struct | SELECT named_struct("a", 1, "b", 2, "c", 3) | struct> | +| org.apache.spark.sql.catalyst.expressions.CreateNamedStruct | struct | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.CsvToStructs | from_csv | SELECT from_csv('1, 0.8', 'a INT, b DOUBLE') | struct> | | org.apache.spark.sql.catalyst.expressions.Cube | cube | SELECT name, age, count(*) FROM VALUES (2, 'Alice'), (5, 'Bob') people(age, name) GROUP BY cube(name, age) | struct | | org.apache.spark.sql.catalyst.expressions.CumeDist | cume_dist | N/A | N/A | @@ -171,7 +172,7 @@ | org.apache.spark.sql.catalyst.expressions.MapEntries | map_entries | SELECT map_entries(map(1, 'a', 2, 'b')) | struct>> | | org.apache.spark.sql.catalyst.expressions.MapFilter | map_filter | SELECT map_filter(map(1, 0, 2, 2, 3, -1), (k, v) -> k > v) | struct namedlambdavariable()), namedlambdavariable(), namedlambdavariable())):map> | | org.apache.spark.sql.catalyst.expressions.MapFromArrays | map_from_arrays | SELECT map_from_arrays(array(1.0, 3.0), array('2', '4')) | struct> | -| org.apache.spark.sql.catalyst.expressions.MapFromEntries | map_from_entries | SELECT map_from_entries(array(struct(1, 'a'), struct(2, 'b'))) | struct> | +| org.apache.spark.sql.catalyst.expressions.MapFromEntries | map_from_entries | SELECT map_from_entries(array(struct(1, 'a'), struct(2, 'b'))) | struct> | | org.apache.spark.sql.catalyst.expressions.MapKeys | map_keys | SELECT map_keys(map(1, 'a', 2, 'b')) | struct> | | org.apache.spark.sql.catalyst.expressions.MapValues | map_values | SELECT map_values(map(1, 'a', 2, 'b')) | struct> | | org.apache.spark.sql.catalyst.expressions.MapZipWith | map_zip_with | SELECT map_zip_with(map(1, 'a', 2, 'b'), map(1, 'x', 2, 'y'), (k, v1, v2) -> concat(v1, v2)) | struct> | @@ -186,7 +187,6 @@ | org.apache.spark.sql.catalyst.expressions.Murmur3Hash | hash | SELECT hash('Spark', array(123), 2) | struct | | org.apache.spark.sql.catalyst.expressions.NTile | ntile | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.NaNvl | nanvl | SELECT nanvl(cast('NaN' as double), 123) | struct | -| org.apache.spark.sql.catalyst.expressions.NamedStruct | struct | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.NextDay | next_day | SELECT next_day('2015-01-14', 'TU') | struct | | org.apache.spark.sql.catalyst.expressions.Not | ! | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Not | not | N/A | N/A | diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out index 3fcd132701a3f..d41d25280146b 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out @@ -272,7 +272,7 @@ struct= 0)):bigint> -- !query SELECT 'foo', MAX(STRUCT(a)) FILTER (WHERE b >= 1) FROM testData WHERE a = 0 GROUP BY 1 -- !query schema -struct= 1)):struct> +struct= 1)):struct> -- !query output 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 7bfdd0ad53a95..50eb2a9f22f69 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 @@ -87,7 +87,7 @@ struct -- !query SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1 -- !query schema -struct> +struct> -- !query output 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 f294c5213d319..3b610edc47169 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 @@ -83,7 +83,7 @@ struct -- !query SELECT ID, STRUCT(ST.C as STC, ST.D as STD).STD FROM tbl_x -- !query schema -struct +struct -- !query output 1 delta 2 eta diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out index ed7ab5a342c12..d046ff249379f 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out @@ -85,7 +85,7 @@ FROM various_maps struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve 'map_zip_with(various_maps.`decimal_map1`, various_maps.`decimal_map2`, lambdafunction(named_struct(NamePlaceholder(), k, NamePlaceholder(), v1, NamePlaceholder(), v2), k, v1, v2))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,0), decimal(36,35)].; line 1 pos 7 +cannot resolve 'map_zip_with(various_maps.`decimal_map1`, various_maps.`decimal_map2`, lambdafunction(struct(k, v1, v2), k, v1, v2))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,0), decimal(36,35)].; line 1 pos 7 -- !query @@ -113,7 +113,7 @@ FROM various_maps struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve 'map_zip_with(various_maps.`decimal_map2`, various_maps.`int_map`, lambdafunction(named_struct(NamePlaceholder(), k, NamePlaceholder(), v1, NamePlaceholder(), v2), k, v1, v2))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,35), int].; line 1 pos 7 +cannot resolve 'map_zip_with(various_maps.`decimal_map2`, various_maps.`int_map`, lambdafunction(struct(k, v1, v2), k, v1, v2))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,35), int].; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index 6403406413db9..da5256f5c0453 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -87,7 +87,7 @@ struct> +struct> -- !query output From 452594f5a43bc5d98fb42bf0927638d139f6d17c Mon Sep 17 00:00:00 2001 From: Prakhar Jain Date: Tue, 26 May 2020 14:13:02 +0900 Subject: [PATCH 07/14] [SPARK-31810][TEST] Fix AlterTableRecoverPartitions test using incorrect api to modify RDD_PARALLEL_LISTING_THRESHOLD ### What changes were proposed in this pull request? Use the correct API in AlterTableRecoverPartition tests to modify the `RDD_PARALLEL_LISTING_THRESHOLD` conf. ### Why are the changes needed? The existing AlterTableRecoverPartitions test modify the RDD_PARALLEL_LISTING_THRESHOLD as a SQLConf using the withSQLConf API. But since, this is not a SQLConf, it is not overridden and so the test doesn't end up testing the required behaviour. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? This is UT Fix. UTs are still passing after the fix. Closes #28634 from prakharjain09/SPARK-31810-fix-recover-partitions. Authored-by: Prakhar Jain Signed-off-by: HyukjinKwon --- .../spark/sql/execution/command/DDLSuite.scala | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) 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 10ad8acc68937..e4709e469dca3 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 @@ -1203,14 +1203,24 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } test("alter table: recover partitions (sequential)") { - withSQLConf(RDD_PARALLEL_LISTING_THRESHOLD.key -> "10") { + val oldRddParallelListingThreshold = spark.sparkContext.conf.get( + RDD_PARALLEL_LISTING_THRESHOLD) + try { + spark.sparkContext.conf.set(RDD_PARALLEL_LISTING_THRESHOLD.key, "10") testRecoverPartitions() + } finally { + spark.sparkContext.conf.set(RDD_PARALLEL_LISTING_THRESHOLD, oldRddParallelListingThreshold) } } test("alter table: recover partition (parallel)") { - withSQLConf(RDD_PARALLEL_LISTING_THRESHOLD.key -> "0") { + val oldRddParallelListingThreshold = spark.sparkContext.conf.get( + RDD_PARALLEL_LISTING_THRESHOLD) + try { + spark.sparkContext.conf.set(RDD_PARALLEL_LISTING_THRESHOLD.key, "0") testRecoverPartitions() + } finally { + spark.sparkContext.conf.set(RDD_PARALLEL_LISTING_THRESHOLD, oldRddParallelListingThreshold) } } From 7e4f5bbd8a40011f6f99f023b05f8c15a4a5453d Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 26 May 2020 05:15:51 +0000 Subject: [PATCH 08/14] [SPARK-31806][SQL][TESTS] Check reading date/timestamp from legacy parquet: dictionary encoding, w/o Spark version ### What changes were proposed in this pull request? 1. Add the following parquet files to the resource folder `sql/core/src/test/resources/test-data`: - Files saved by Spark 2.4.5 (https://github.com/apache/spark/commit/cee4ecbb16917fa85f02c635925e2687400aa56b) without meta info `org.apache.spark.version` - `before_1582_date_v2_4_5.snappy.parquet` with 2 date columns of the type **INT32 L:DATE** - `PLAIN` (8 date values of `1001-01-01`) and `PLAIN_DICTIONARY` (`1001-01-01`..`1001-01-08`). - `before_1582_timestamp_micros_v2_4_5.snappy.parquet` with 2 timestamp columns of the type **INT64 L:TIMESTAMP(MICROS,true)** - `PLAIN` (8 date values of `1001-01-01 01:02:03.123456`) and `PLAIN_DICTIONARY` (`1001-01-01 01:02:03.123456`..`1001-01-08 01:02:03.123456`). - `before_1582_timestamp_millis_v2_4_5.snappy.parquet` with 2 timestamp columns of the type **INT64 L:TIMESTAMP(MILLIS,true)** - `PLAIN` (8 date values of `1001-01-01 01:02:03.123`) and `PLAIN_DICTIONARY` (`1001-01-01 01:02:03.123`..`1001-01-08 01:02:03.123`). - `before_1582_timestamp_int96_plain_v2_4_5.snappy.parquet` with 2 timestamp columns of the type **INT96** - `PLAIN` (8 date values of `1001-01-01 01:02:03.123456`) and `PLAIN` (`1001-01-01 01:02:03.123456`..`1001-01-08 01:02:03.123456`). - `before_1582_timestamp_int96_dict_v2_4_5.snappy.parquet` with 2 timestamp columns of the type **INT96** - `PLAIN_DICTIONARY` (8 date values of `1001-01-01 01:02:03.123456`) and `PLAIN_DICTIONARY` (`1001-01-01 01:02:03.123456`..`1001-01-08 01:02:03.123456`). - Files saved by Spark 2.4.6-rc3 (https://github.com/apache/spark/commit/570848da7c48ba0cb827ada997e51677ff672a39) with the meta info `org.apache.spark.version = 2.4.6`: - `before_1582_date_v2_4_6.snappy.parquet` replaces `before_1582_date_v2_4.snappy.parquet`. And it is similar to `before_1582_date_v2_4_5.snappy.parquet` except Spark version in parquet meta info. - `before_1582_timestamp_micros_v2_4_6.snappy.parquet` replaces `before_1582_timestamp_micros_v2_4.snappy.parquet`. And it is similar to `before_1582_timestamp_micros_v2_4_5.snappy.parquet` except meta info. - `before_1582_timestamp_millis_v2_4_6.snappy.parquet` replaces `before_1582_timestamp_millis_v2_4.snappy.parquet`. And it is similar to `before_1582_timestamp_millis_v2_4_5.snappy.parquet` except meta info. - `before_1582_timestamp_int96_plain_v2_4_6.snappy.parquet` is similar to `before_1582_timestamp_int96_dict_v2_4_5.snappy.parquet` except meta info. - `before_1582_timestamp_int96_dict_v2_4_6.snappy.parquet` replaces `before_1582_timestamp_int96_v2_4.snappy.parquet`. And it is similar to `before_1582_timestamp_int96_dict_v2_4_5.snappy.parquet` except meta info. 2. Add new test "generate test files for checking compatibility with Spark 2.4" to `ParquetIOSuite` (marked as ignored). The parquet files above were generated by this test. 3. Modified "SPARK-31159: compatibility with Spark 2.4 in reading dates/timestamps" in `ParquetIOSuite` to use new parquet files. ### Why are the changes needed? To improve test coverage. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running `ParquetIOSuite`. Closes #28630 from MaxGekk/parquet-files-update. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../before_1582_date_v2_4.snappy.parquet | Bin 398 -> 0 bytes .../before_1582_date_v2_4_5.snappy.parquet | Bin 0 -> 660 bytes .../before_1582_date_v2_4_6.snappy.parquet | Bin 0 -> 694 bytes ...timestamp_int96_dict_v2_4_5.snappy.parquet | Bin 0 -> 737 bytes ...timestamp_int96_dict_v2_4_6.snappy.parquet | Bin 0 -> 771 bytes ...imestamp_int96_plain_v2_4_5.snappy.parquet | Bin 0 -> 693 bytes ...imestamp_int96_plain_v2_4_6.snappy.parquet | Bin 0 -> 727 bytes ...e_1582_timestamp_int96_v2_4.snappy.parquet | Bin 494 -> 0 bytes ..._1582_timestamp_micros_v2_4.snappy.parquet | Bin 436 -> 0 bytes ...582_timestamp_micros_v2_4_5.snappy.parquet | Bin 0 -> 767 bytes ...582_timestamp_micros_v2_4_6.snappy.parquet | Bin 0 -> 801 bytes ..._1582_timestamp_millis_v2_4.snappy.parquet | Bin 436 -> 0 bytes ...582_timestamp_millis_v2_4_5.snappy.parquet | Bin 0 -> 761 bytes ...582_timestamp_millis_v2_4_6.snappy.parquet | Bin 0 -> 795 bytes .../datasources/parquet/ParquetIOSuite.scala | 188 ++++++++++++------ 15 files changed, 130 insertions(+), 58 deletions(-) delete mode 100644 sql/core/src/test/resources/test-data/before_1582_date_v2_4.snappy.parquet create mode 100644 sql/core/src/test/resources/test-data/before_1582_date_v2_4_5.snappy.parquet create mode 100644 sql/core/src/test/resources/test-data/before_1582_date_v2_4_6.snappy.parquet create mode 100644 sql/core/src/test/resources/test-data/before_1582_timestamp_int96_dict_v2_4_5.snappy.parquet create mode 100644 sql/core/src/test/resources/test-data/before_1582_timestamp_int96_dict_v2_4_6.snappy.parquet create mode 100644 sql/core/src/test/resources/test-data/before_1582_timestamp_int96_plain_v2_4_5.snappy.parquet create mode 100644 sql/core/src/test/resources/test-data/before_1582_timestamp_int96_plain_v2_4_6.snappy.parquet delete mode 100644 sql/core/src/test/resources/test-data/before_1582_timestamp_int96_v2_4.snappy.parquet delete mode 100644 sql/core/src/test/resources/test-data/before_1582_timestamp_micros_v2_4.snappy.parquet create mode 100644 sql/core/src/test/resources/test-data/before_1582_timestamp_micros_v2_4_5.snappy.parquet create mode 100644 sql/core/src/test/resources/test-data/before_1582_timestamp_micros_v2_4_6.snappy.parquet delete mode 100644 sql/core/src/test/resources/test-data/before_1582_timestamp_millis_v2_4.snappy.parquet create mode 100644 sql/core/src/test/resources/test-data/before_1582_timestamp_millis_v2_4_5.snappy.parquet create mode 100644 sql/core/src/test/resources/test-data/before_1582_timestamp_millis_v2_4_6.snappy.parquet diff --git a/sql/core/src/test/resources/test-data/before_1582_date_v2_4.snappy.parquet b/sql/core/src/test/resources/test-data/before_1582_date_v2_4.snappy.parquet deleted file mode 100644 index 7d5cc12eefe04f18f8a2ba99db22326dc0399a80..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 398 zcmaKp!Arw16vneSMv)y&X&{FZ+}NOo*0F9aUc9NN;YkpYbWL^Gc3aX;Wc2F);J@lc z-`oUGULNo9eeeCgklE#JKoB`0jz=&w9bz%3{r>U!wXZfB{Fx9!$EO(TQEO)*n0i-T zvy%Gd(afJ!AqhqV_psAWWust)0sTc|uWJig!Z zQWiUD7ZZeo@i3Kjp35W$9Tt_xe^2sw+cXaia&8X5vqHNYrOjelx`7CM5x9e-UgW7O tr%{?svnU#!C!xy1a5PH&&<`Uu_UBPDi6(xck~A1n#^8C^Ere$96W`?#bIbq$ diff --git a/sql/core/src/test/resources/test-data/before_1582_date_v2_4_5.snappy.parquet b/sql/core/src/test/resources/test-data/before_1582_date_v2_4_5.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..edd61c9b9fec8239636030cbda2789e223fc6320 GIT binary patch literal 660 zcmb7CJxjw-6uk+tWayB=7ZS)&9<(ULkXTHc(#1{0p}303OY^G1^eagfDP8@nTO9;v z2S@*vf5Am>nrd|sgq$1ByYHObb4hD^-ysr_NpX*eL;{eCmyhos1c*XbO)L^A5ZTbA z>@=?`to>jBAR8i9MF0h5d!&`gU?IEzC0jBSRA#v>bG|}cBW@74h&#j-kqL;XH+PE3 zfW@arN&AF%8O}+>vZzTx*iOsEoU9BY7WR;-if-s;MK(~lB^M+;zehw>QKy1B6Do4^ zk371GnI$3uX2@PR=#!9pK|$B@&&30Tkprm z78@|kV$EY&$uP1o-*CiJEAF3LU7oUlrA&Je#x%VcaN3|r8jsp3HEAd0QIOEa!I<_~ zH)~~Uz{cVt3OTagD2kXLVIPIuq+g!2F`i7!Z|!_*{}UfhK-*o6yU~qHXFN{Aey`+M zj%_)mHGdRFL5bD9pi}R7UUk!VdB=6D)xdUb*W+7u+w*H)&Gxw;IP1D;0N?{}0zeD@ F$4_X9%2WUV literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/test-data/before_1582_date_v2_4_6.snappy.parquet b/sql/core/src/test/resources/test-data/before_1582_date_v2_4_6.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..01f4887f5e994c77fc587378e7f3a19868745681 GIT binary patch literal 694 zcmb7CJ&W5w5FH8G2+oCytdKy3SvX^ZvPgC;TgFYg<8WcF4I!+obs{2vNpiwr+~(8i z?uvudDXzSqcg@|b<-`erKtOMorMY$ggG^MYH~%7zw~ zOF;)TFS7Ndpe~&zB8_u8c(I{#K9eQ6C@dTrQ7Dkj*J;Y56zeDyF8%3A2b=BI{iWUS z_5Z{#w_u$g#(g@=>W6%FpZOm1>&x>~f{-`Kkr?F&Zc*)x)jy3hxP1{GHi+KLvv~mwzG~-M zt%2lL!BR7mptO0piJ*rt4Z36oh$`nfcy)-}~O=mV{E>0S06WwOUpXm0ELjNx)0O+=gv#CJM_E90H$2Bj&vn|Q%2Hflzu!})3 zD8{XhF=klJg|^jHM#=5V$^xW@YyC!{)q4BsCsD2O1Eb7E@$zXoc_A}_Vx)anWm=p} zs%E*9ry`jFH7(n6d%8{9)}hwcttN-4)TCb(`c3KS<88OwyMK7MTy(4vweDyA0qA{K zF14B>)?)8f`dLA6jDD`(qCZ;H#B;{x#d9ulfwuVCA%7){GMUfV&OSfj0kFL*8Rc2Z zu^DFzS@B|i&NEqwSX6?2P?k>hCYR30DXU^JsT|jtCNhmnXY_dK%tTg0b*S1nW-!2g8#Uu!oz%?`1`I5gg5FH8E_@IOOv0DjbY8Gm2P=!=Cie-E$4@qvoFu544>YjCW=Qs-==<0J5o_@^2?uWRrEgJ4jMmvyag;52Jm z6LX@P{X?zHdGYmAIs76rPAcf&wQ5wNWGI*ElRV|g2&isarroSrq-yTfplU8LM5Q|Y zt#5@6Q$%?jz^vOmf0GrBGf|Yuc+~J5 z&vm>;E1V>0)ZpD9io0W-+bTu%;fap(~to9WRXLkxR|v`$72q>9jfps(i)Qjlmx{EI;J(dTX+AOQbf+p*J_YJ>kNf?ON80=d+8w^u5SzO6 zyG=h^{myusbbIv=Ztfu+4unAcuhw;wb@x@59~S*+VR4MuH{|qNi@Lyl#%A?PsVapa za>XRib=~B%Dcd=ahcW`T|C^#YDSDeXth7ysNYriN@6Pi4nfY za$Tg27(H8vsVXl@7pH`UjnL$!Zj34`+?vjdLL~)?@N4ahOJgxwESLV3!Rl=P!{PZ7 z*ptoR56Zf?A?<}S3}o2bOXhiz_S7&=vtbs;gM%c}SriQhX%GcbtdD|8oE*o;L86m1 R+~>XrfDi2A1)Sn9{R=(^uO$Eg literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/test-data/before_1582_timestamp_int96_v2_4.snappy.parquet b/sql/core/src/test/resources/test-data/before_1582_timestamp_int96_v2_4.snappy.parquet deleted file mode 100644 index 13254bd93a5e6c54a13beb4314757d76b98117e5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 494 zcmaixPfNov7{=dbrS!0ah$#(3ECt5~wX}|P>+s@D#KZ6^BI%lHvE6Eu`iHS^;dk@m z(GOxeCb)}U65i+c=6#+dv-9f^>)6Kh3L~robUk>Tu=hiN{f{HSfcNl$*CK3SliJ<`2AN7y&kpk&UDP3lU0a&!pHOBSSYyt72p! z>r=D7w4QmkYrI?i6N7l_Dtc=Nn+>3yihxjiu&nQWRxuUG-(>T}Q{k(SPYsuTy?pUY zE?LSXvu{?ee5<%Sb%m^(xpX}@%XyIscdF2Wl`WD&$=f!Wmt06zR!UklMZuCnwWzb) zbAP6~)75%?y9WDm>v>Yv&NHt?zFat=9|nHt^phqpQio0AG@E2`JUmSzo<-4cm ztAFr>n#y1R0Gl`hLf!7Xy60jc44ePf;A&ZL(`Q*r&8zt{FQpc#(83+Bn0{GEHe^ay z^;9#TWw|U;#fD}y6Xj;4m9e*LIxnTtqFfkjRu_dx3bSIe(r3F{*>JU9PuAdGZr@-X0`*GcNUNIfx#(`=B%aql#VWEMreUK&I}6wC8q8Ylg@A0#qK!!DuJ MJaKFRjLd)i023L1p8x;= diff --git a/sql/core/src/test/resources/test-data/before_1582_timestamp_micros_v2_4_5.snappy.parquet b/sql/core/src/test/resources/test-data/before_1582_timestamp_micros_v2_4_5.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..62e6048354dc1026f86af85a5c8b29eb492dfaf8 GIT binary patch literal 767 zcmbV~zl+pB6vyA>;*#QufHxt*6zjk(w-7_(a=FcV2sUzv#c`Ez9x~a?YB1T&C0Rwo zf`urEomg2|SY2gf=Z{LTwiB%HudvXUY}N%=8zIb_`M#IWy!TD^uN=5krYhaIP9-V> zC?>prb8_S8#ZL;AY~do)7FA4LS!9+w4YvNGLWPlVRSBqwc~0mPQh{IrI(+)@6tDOL zw4x&7>RtnCx`K1%mzXH)<-zGA4BxwJFELyq=T9)a`gZvvQTjjL-u#MmXn0gIcK6ov zlxKHu<-ca$P1pND2=Ca%!3$<@NHqS1(*%W+06P4sIC1E!qE)$Yfa-9 zlB|7CWO+0hw_V$HY`47?Ortn#^S&RB`Xk@(T?{-i^1NOzbUerN#kMo_gB^dz2}BUO RUBfZ~@PV`V8TRql{{_RV;WPjM literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/test-data/before_1582_timestamp_micros_v2_4_6.snappy.parquet b/sql/core/src/test/resources/test-data/before_1582_timestamp_micros_v2_4_6.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d7fdaa3e67212c1991ba75219827b490c127f43e GIT binary patch literal 801 zcmbW0zl+pB6vyA>az~0QQrv_DQ>=koZXphvTQ|2^55Yzbu{f?0&O;`fSq&z;xg@LH zg#`;y5IeE5u&}zy#?Bv=U~MN@-(O*&FWIaMt~NrLH}icjpLy>avVZNsp)ytJ#!V_w z89*`N{hPB}M=yR-C}kTLnXXaA)RlQ=vD0AdFDjH630IYXikRnwJ|QIt=Agr;r{{RZ zAD|@_5!d%>P~8<=D858QSuYOGA7S|3T6u}#3b}ZO;nlaRABocY@%Hvtq(jZ4lCitD zo+UiJe)&9hEdydq=bn zq^ayKI$!MmVTNI&fijf<$1ejw^E0W~P_k#&R`6$%_E^GXNni4!qA#gnn23c6t9@G0 zR4{B^AE$R2PdLuXseZeJjh;$so6EYZ!_s$gKWhVU2iPrb;6*Q5WW6EV7)(2}+|$6mdVoB79re)*Nj0 zi|N$<4XDij55mzD^c!{Km*TWhPug&p(`HU%%b$c%(BM5U81{yq*V*=6F?8KdCuqBE e*Aw0L!1H&!owhIhz-bw_34jlD@k8z7&+!XBE9?dU literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/test-data/before_1582_timestamp_millis_v2_4.snappy.parquet b/sql/core/src/test/resources/test-data/before_1582_timestamp_millis_v2_4.snappy.parquet deleted file mode 100644 index e9825455c20158d1d0cbb8add20ca5833d5be9cc..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 436 zcmZ{hK}*9h7>2*bF?tXWLP`TUl)%OYHMGuk>+s@DJc-vKlCG(iwOiMu${70_9{hd& zkD1BFE_w+s@AtiV-jK(;rw{|C*z++mqK$3p+G-B#;Cr__DF=FgxBzf*gUoqm9U~YK z|NdN6qMpB~*_$i{Mu1x#`?7sM}xGuUu5Zu=#Hdu2uy%eO8Usyk36rQfiS3E!^pb>2)R9gehG&GtGRK z<+4Z>o0!o;l>3oZ#y+g+yp&3dvNG19DGHGkX2oQs&yKgU$!5EK-h%sZU|yBA_bzLd zFBe|O!+?igFKO~3^~5+%vvC&3gX<)cSriQhX%GcbEQi4?PDb%4NMw?ReL|^u;?x3o HH2?JjJC=Zu diff --git a/sql/core/src/test/resources/test-data/before_1582_timestamp_millis_v2_4_5.snappy.parquet b/sql/core/src/test/resources/test-data/before_1582_timestamp_millis_v2_4_5.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..a7cef9e60f13425560f516f4ffb32f0bb4b1831a GIT binary patch literal 761 zcmbV~F^kkd6vrnqyG(IKP$y&{#WHfs9mJ5hTz0cuBZ!D79JV4d+01IVWH*;$Z6xVPZC`9Qts>*;1a_859qM+mLkzly+Xij&h;EtIN6k9Zx_Ezm==JI!W1udK z&LBF1&Z`$f-OpOXMr~?0Wo!7UZvE6CSkX7eP}4VDL><7Q8f%?huuL%Q-D&yA;{}Ia zy}E5r9%UZ%e>4-yE2mGqTqvG!#qAH;oT-ySupyJGoQ)K7**FzhBH8f%oK1LMg{m<@ zV40}WTu8<90<6g_%XpkY5_~OOwgR)^e6etU1M1QK2jRg2*|(PmznYhwV^PXPZ96vh3h@5dref?eu52*HCc!EZQ# GOaBW#&F0kr literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/test-data/before_1582_timestamp_millis_v2_4_6.snappy.parquet b/sql/core/src/test/resources/test-data/before_1582_timestamp_millis_v2_4_6.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..4c213f4540a735c041a85fb12bc90810a3f83eb1 GIT binary patch literal 795 zcmbW0L5tHs6vtm;y5z8jAWldihZ5Pc3vo!cHf_pY1X)BB7Ox^QX{H)BX|tJBS!(s* zWe<9B56d3JFX3?y9t1yxSMloA+xjMLEm*w>;h&lJUVd-hKg_|c2LaKDL9X2+D$xLR zM;yYj|LyXUfDC&m8rdMaW9iGxYGuOqU!ou}DypG?ftWW6c_BHlMc4*pBVBbcbkmN88Ztc^y$r!~Ce27fMUtTOE3}`iBj+ zht_FC1)N;I09v1wnhlaYZ(GApRP(2TNl9O`p{B1z!FCV}HP*biVj5t3mNhP)P*yOU zmDXd^XS@{Ycw{wp=}j7fwX;?sjxvf49Zh&e%kguX^NJ;`V%7&`>Q=`Ecl)lW%E_Q| zJ$IP$ED>)1;nW?mT&ARvm{=56Y0gE(@&ZGnNtUrV!y^2ed+rkK_NTL%_Zv`~{~v@$ zGqCP14t^sqn~O=C0S$Z_G`HhPnk7xvi;`h)7)9;fIOM}HY_}6X^uvgE{6Q3VqplzG TI0<%a&jG*(I{2dw@LK!=c(3uN literal 0 HcmV?d00001 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index f075d04165697..79c32976f02ec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources.parquet +import java.nio.file.{Files, Paths, StandardCopyOption} import java.sql.{Date, Timestamp} import java.time._ import java.util.Locale @@ -45,7 +46,7 @@ import org.apache.spark.{SPARK_VERSION_SHORT, SparkException, SparkUpgradeExcept import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{InternalRow, ScalaReflection} import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeRow} -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils} import org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -875,81 +876,152 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } + // It generates input files for the test below: + // "SPARK-31159: compatibility with Spark 2.4 in reading dates/timestamps" + ignore("SPARK-31806: generate test files for checking compatibility with Spark 2.4") { + val resourceDir = "sql/core/src/test/resources/test-data" + val version = "2_4_5" + val N = 8 + def save( + in: Seq[(String, String)], + t: String, + dstFile: String, + options: Map[String, String] = Map.empty): Unit = { + withTempDir { dir => + in.toDF("dict", "plain") + .select($"dict".cast(t), $"plain".cast(t)) + .repartition(1) + .write + .mode("overwrite") + .options(options) + .parquet(dir.getCanonicalPath) + Files.copy( + dir.listFiles().filter(_.getName.endsWith(".snappy.parquet")).head.toPath, + Paths.get(resourceDir, dstFile), + StandardCopyOption.REPLACE_EXISTING) + } + } + DateTimeTestUtils.withDefaultTimeZone(DateTimeTestUtils.LA) { + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> DateTimeTestUtils.LA.getId) { + save( + (1 to N).map(i => ("1001-01-01", s"1001-01-0$i")), + "date", + s"before_1582_date_v$version.snappy.parquet") + withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> "TIMESTAMP_MILLIS") { + save( + (1 to N).map(i => ("1001-01-01 01:02:03.123", s"1001-01-0$i 01:02:03.123")), + "timestamp", + s"before_1582_timestamp_millis_v$version.snappy.parquet") + } + val usTs = (1 to N).map(i => ("1001-01-01 01:02:03.123456", s"1001-01-0$i 01:02:03.123456")) + withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> "TIMESTAMP_MICROS") { + save(usTs, "timestamp", s"before_1582_timestamp_micros_v$version.snappy.parquet") + } + withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> "INT96") { + // Comparing to other logical types, Parquet-MR chooses dictionary encoding for the + // INT96 logical type because it consumes less memory for small column cardinality. + // Huge parquet files doesn't make sense to place to the resource folder. That's why + // we explicitly set `parquet.enable.dictionary` and generate two files w/ and w/o + // dictionary encoding. + save( + usTs, + "timestamp", + s"before_1582_timestamp_int96_plain_v$version.snappy.parquet", + Map("parquet.enable.dictionary" -> "false")) + save( + usTs, + "timestamp", + s"before_1582_timestamp_int96_dict_v$version.snappy.parquet", + Map("parquet.enable.dictionary" -> "true")) + } + } + } + } + test("SPARK-31159: compatibility with Spark 2.4 in reading dates/timestamps") { + val N = 8 // test reading the existing 2.4 files and new 3.0 files (with rebase on/off) together. - def checkReadMixedFiles(fileName: String, dt: String, dataStr: String): Unit = { + def checkReadMixedFiles[T]( + fileName: String, + catalystType: String, + rowFunc: Int => (String, String), + toJavaType: String => T, + checkDefaultLegacyRead: String => Unit, + tsOutputType: String = "TIMESTAMP_MICROS"): Unit = { withTempPaths(2) { paths => paths.foreach(_.delete()) val path2_4 = getResourceParquetFilePath("test-data/" + fileName) val path3_0 = paths(0).getCanonicalPath val path3_0_rebase = paths(1).getCanonicalPath - if (dt == "date") { - val df = Seq(dataStr).toDF("str").select($"str".cast("date").as("date")) - + val df = Seq.tabulate(N)(rowFunc).toDF("dict", "plain") + .select($"dict".cast(catalystType), $"plain".cast(catalystType)) + withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> tsOutputType) { + checkDefaultLegacyRead(path2_4) // By default we should fail to write ancient datetime values. - var e = intercept[SparkException](df.write.parquet(path3_0)) + val e = intercept[SparkException](df.write.parquet(path3_0)) assert(e.getCause.getCause.getCause.isInstanceOf[SparkUpgradeException]) - // By default we should fail to read ancient datetime values. - e = intercept[SparkException](spark.read.parquet(path2_4).collect()) - assert(e.getCause.isInstanceOf[SparkUpgradeException]) - withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE.key -> CORRECTED.toString) { df.write.mode("overwrite").parquet(path3_0) } withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE.key -> LEGACY.toString) { df.write.parquet(path3_0_rebase) } - - // For Parquet files written by Spark 3.0, we know the writer info and don't need the - // config to guide the rebase behavior. - withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ.key -> LEGACY.toString) { - checkAnswer( - spark.read.format("parquet").load(path2_4, path3_0, path3_0_rebase), - 1.to(3).map(_ => Row(java.sql.Date.valueOf(dataStr)))) - } - } else { - val df = Seq(dataStr).toDF("str").select($"str".cast("timestamp").as("ts")) - withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> dt) { - // By default we should fail to write ancient datetime values. - var e = intercept[SparkException](df.write.parquet(path3_0)) - assert(e.getCause.getCause.getCause.isInstanceOf[SparkUpgradeException]) - // By default we should fail to read ancient datetime values. - e = intercept[SparkException](spark.read.parquet(path2_4).collect()) - assert(e.getCause.isInstanceOf[SparkUpgradeException]) - - withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE.key -> CORRECTED.toString) { - df.write.mode("overwrite").parquet(path3_0) - } - withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE.key -> LEGACY.toString) { - df.write.parquet(path3_0_rebase) - } - } - // For Parquet files written by Spark 3.0, we know the writer info and don't need the - // config to guide the rebase behavior. - withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ.key -> LEGACY.toString) { - checkAnswer( - spark.read.format("parquet").load(path2_4, path3_0, path3_0_rebase), - 1.to(3).map(_ => Row(java.sql.Timestamp.valueOf(dataStr)))) - } + } + // For Parquet files written by Spark 3.0, we know the writer info and don't need the + // config to guide the rebase behavior. + withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ.key -> LEGACY.toString) { + checkAnswer( + spark.read.format("parquet").load(path2_4, path3_0, path3_0_rebase), + (0 until N).flatMap { i => + val (dictS, plainS) = rowFunc(i) + Seq.tabulate(3) { _ => + Row(toJavaType(dictS), toJavaType(plainS)) + } + }) } } } - - withAllParquetReaders { - checkReadMixedFiles("before_1582_date_v2_4.snappy.parquet", "date", "1001-01-01") - checkReadMixedFiles( - "before_1582_timestamp_micros_v2_4.snappy.parquet", - "TIMESTAMP_MICROS", - "1001-01-01 01:02:03.123456") - checkReadMixedFiles( - "before_1582_timestamp_millis_v2_4.snappy.parquet", - "TIMESTAMP_MILLIS", - "1001-01-01 01:02:03.123") - - // INT96 is a legacy timestamp format and we always rebase the seconds for it. - checkAnswer(readResourceParquetFile( - "test-data/before_1582_timestamp_int96_v2_4.snappy.parquet"), - Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123456"))) + def failInRead(path: String): Unit = { + val e = intercept[SparkException](spark.read.parquet(path).collect()) + assert(e.getCause.isInstanceOf[SparkUpgradeException]) + } + def successInRead(path: String): Unit = spark.read.parquet(path).collect() + Seq( + // By default we should fail to read ancient datetime values when parquet files don't + // contain Spark version. + "2_4_5" -> failInRead _, + "2_4_6" -> successInRead _).foreach { case (version, checkDefaultRead) => + withAllParquetReaders { + checkReadMixedFiles( + s"before_1582_date_v$version.snappy.parquet", + "date", + (i: Int) => ("1001-01-01", s"1001-01-0${i + 1}"), + java.sql.Date.valueOf, + checkDefaultRead) + checkReadMixedFiles( + s"before_1582_timestamp_micros_v$version.snappy.parquet", + "timestamp", + (i: Int) => ("1001-01-01 01:02:03.123456", s"1001-01-0${i + 1} 01:02:03.123456"), + java.sql.Timestamp.valueOf, + checkDefaultRead) + checkReadMixedFiles( + s"before_1582_timestamp_millis_v$version.snappy.parquet", + "timestamp", + (i: Int) => ("1001-01-01 01:02:03.123", s"1001-01-0${i + 1} 01:02:03.123"), + java.sql.Timestamp.valueOf, + checkDefaultRead, + tsOutputType = "TIMESTAMP_MILLIS") + // INT96 is a legacy timestamp format and we always rebase the seconds for it. + Seq("plain", "dict").foreach { enc => + checkAnswer(readResourceParquetFile( + s"test-data/before_1582_timestamp_int96_${enc}_v$version.snappy.parquet"), + Seq.tabulate(N) { i => + Row( + java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123456"), + java.sql.Timestamp.valueOf(s"1001-01-0${i + 1} 01:02:03.123456")) + }) + } + } } } From 8ba2b47737f83bd27ebef75dea9e73b3649c2ec1 Mon Sep 17 00:00:00 2001 From: Xingcan Cui Date: Tue, 26 May 2020 14:27:42 +0900 Subject: [PATCH 09/14] [SPARK-31792][SS][DOCS] Introduce the structured streaming UI in the Web UI doc ### What changes were proposed in this pull request? This PR adds the structured streaming UI introduction to the Web UI doc. ![image](https://user-images.githubusercontent.com/1452518/82642209-92b99380-9bdb-11ea-9a0d-cbb26040b0ef.png) ### Why are the changes needed? The structured streaming web UI introduced before was missing from the Web UI documentation. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? N.A. Closes #28609 from xccui/ss-ui-doc. Authored-by: Xingcan Cui Signed-off-by: HyukjinKwon --- .../img/webui-structured-streaming-detail.png | Bin 0 -> 179135 bytes docs/web-ui.md | 28 ++++++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 docs/img/webui-structured-streaming-detail.png diff --git a/docs/img/webui-structured-streaming-detail.png b/docs/img/webui-structured-streaming-detail.png new file mode 100644 index 0000000000000000000000000000000000000000..f4850523c5c2fe165eb6c1290ccc0dfe0c107b71 GIT binary patch literal 179135 zcmeFZXE${BlS|E}}pd^=yxx&G_Al-ISES?{}^_bK;t-z)3|P>ucy>lG?0DtZm| zXS!5WG%zYETJ}pcl)vbYY1mPIob%LG`-`fykA0akxM;7WqeMkj7DIRRnwm1c?5=L) zNkzrPd-iosL-+m$71e33#xo^7AM>^2K{s}_j3vM6Q%llY9OTIIT_AVnzWX`o@n3Jw zU7o1AcrNi>yY9vXYQ;|&#=qI_C~qZbmpr@XNp~Sz(?ILA2ua9e?755>TT`N7wR~a2SLv4JVH50i~NPi~2CL0WV;9BtzEemM`7t+2Hf-5d%XmXL>s7ZV{qod{W!B)!npQZfKG=a@RyDB54Fg9=3 z!_$Mv-y)MOdZf@BPh?K^@*^!^No}%ek`u7!&$B5-@-Z@+%~c7|xz*#bnr?}_ zHIs(15gA7gtf$#FJmvpR;hjNi1~m*x?CBb}`CI_(inZktPfdX@4SM8s57w`JU0 ziU+P9*YacxGCMfzN17J9JeGK!B4-qrGI(|cYg`-u;mEWvNFD(Rku zg1PH}fC1;_|BNITeVT<{PxL?Nww{ju5shL4J9f^=N~B~+TaOYtCz!~;HO%^IF+ab7 z=+f*$3olKQY}+3D?035#m)as&e)QA%kI7nE?8+<+fA+)FUzK*)qWLmvD@sC+ zi9`{dJT_rGFX1~IEG&r9=DHj&$f(WMo9sYH+((hlAx68*G4H5)kQq(7_(3r6x$UMI z4oT(!uSjyyLk;V>;XR#=E$Ex&-!31+2PUJ5O*Oer4-tO26Kdm=ev z&jkaeUjQQ4*flY^`#fvEVG z{!s)SqYRd~S*2iLbC22*x3DY4h<+XW7{?jy=!KSVy2p$jt_N-Vpf75{m*BFDis6+g$3T{ z^~^(ndHyGdzB|a$3!+kkrbU!ZIldF!YoAtb+Q^N^De#5#2GFB6!s0wmbYCS(d6vE% z@_EQe9JX0s`8z#8R-t6(M{L~euo+#olx%Ut@NC2RD6Y|;ZTlvmAo}jkrN4WoppHm) zaiq;9 zXwh+Q@p@Ts$sCgZe)oM|geLJ9c{l$!Yy|e8xg&)9)}xfKD8@N#FQX%xp!Knxk*)mJ zsAu)mz^zISH(Hk$xqPHR_1Z%v>_L`!LWEM9=cGgZT0l0&j^YJkQXc+s?ug_j8dzfc zWV9^*ZF9ICV+c@W)b{Y1sNu}=x2w$gTv1->dATiv(Ap%XyU=p(IDl6c-g6qfqil4f zP?9D)*8s`vZX6d&>0minC|O)eYd@w(mIf*a^Fx7D9s=$b;mHGU$;yL4-kHqKjixFz}Qk|Db_w}FD| zlNPY`Uv`e8<4L!`zU6#kIj&g`c&esED4Uf0xPHpxO9}ShP4lD3DrgiuBrf+*tZ7iv z-0wEnIMxpn0M!QTuDhbFR)*~b1cbjsyt|qM^oMp|2R^2_762l$>O(r%@ihn%ESIZ8 z$Nc<+VKML`EOC2r_KnJV?(V_qwRs+UJn_e}n8)O{FGdOOrd#UNa$V9vDeke)?KCU5 zm=It<@wO3Wvf5R*!djt3XyUwkv}GotC55hhr>7y)U?mlp`f*{cC)F9x(l{EE1+tik zJf;IkcvSQAHMVA<`=k;k&*eOBTTfzes)644FM2mP9Z z&F=6rYLI-}GQVA`CbLCpqQKOkaiv^jv*IFdHZ^ zOe=e?1fNkz_ZO^L#4D;`KtVNAX;zFO__yqx*<%YW%H;9#QbULHsz08$8j*g*5brpO?)lQ-ZKvg_A zner%hQ-x@;BAC2+XU!M0Rn>M2?`3>J{Yw0CAJsxcjbcacGHQ9=T;{pfzT)!!OUH=V ziOZBnO$B!Zro+l<_CBau30}D!P8+_Zfi;kG^J50Dc*FV|7BgFa*GTATbhEIMqm<)s zl2mV#q-`JiNdai+U>{&h`$g&9?g0ZHE@?YU*bfwkk>yyfX(%E*Bf=z{At3>I(UW&# zQEoTne`38@NRXNslx}$z`8s$@yEjqc#=XI?hE+z%zgLO=bC)m5X)^MGnZcS|Uk7_T zX@c-4fz+@=3w%W|@B&fKX7^>$x3_f%0!s|$qf+SIA@7q~l+I`yyxY{bo zbHi$xxys7v@FG|bv$fC~C|*Tq@jKYf2; z+~LW+kx+ipm!<^GH7kT2en1i=8n=|-^7-2ZN^tZmXflg>*!*S412?L*En`QHP8UBN zJJnF^9xRoTethw!N3Ne2fX=&#>3JV?-Uc5|T6oj8gYOrPKS)~NN)}m~XB_x||J~@S z0HDrw<5Ko;{R(>bC;#{HS~D6Ds3e^`*PsM1a#}pTXz}i@x_w+u_SGQ%3VV52MhFm8o;sxAhZvIC zNRZ7eCHSt(c_%h8fTTe62mR2XV4&Yi{4vH~{n0eaNV0#$qlvKX)_G7T8{X?3=+M%$ zzUExo*)taFpCc*HL#m_yR^#g?HatxYYsl$M)qvk?;HKM+LKf^z5MB=mEKIwxB}^zy z09;=WEnA7avRw|eD*Tg6&D^f2R^ktZ%=9$|l`GS(f6|`T*#qu9S=u=fCSKwa^!i)% z>+rNjT>yX3jBw%>4QvST925+6c+Ye;lN}%z<#!Gg6i|1plo~lGb1!Jzi(-NZ!4A59 z!9>mM+l{Mq$B|X1WPHT4I zxE-3{d5$jw!sJRb9I(%&Z+J*Oh_DF_da?LHHJNwq==&>6KxKS6z3!2@0IT|VgrqV- zc>p!&Q~4+&;j&D^^>>b$A3e@ioOF_w67gf4-(s?w_&&`k&=^V0i%?zJAxhTRP*p%l?f|AD5t#LMs`t@xV3Y5tAH3nH8*= zHc=}70=;Ev*Gf)V4*{~|#0QJ}+{^DUBDM7S9>&{*+}v6at1ts~RRz^QWb9>nB7k`S z#qCu0sU6-1f3f$H$()SIAgcOd2ekqdM(bPXMj|Cb+3TBL8m3OD0feItkXnKfjc>&C z2K}Ga(F1CcZWghz$bgFlE&+`ccT`lG8|krwvp1Uvc-WnEjOLCG8Z?>uQ}r@@QJ{ro z1Cj4<-W2ur6O-THvC+vx113dE=1n6{gN1S-cOcB*Er^Gc?`pV^_nm{X3I&mf_4P*B z3BT>S-*!{ZW^cSAmD&L6+cP!qi3hbUma_e~oMe>_d}W(LhP4JXAW~>GSqIjC3!Xqb z#~oK?NQQHl<8F95>z=dQP_2r^hO-t^&@49a8DF!8_Dv+&G*KjX(gJ1aqPW^HSe*qL zxFKYrxo-cLot^4Ugl!e|f%xjwj0Q`lNo=SC8SMpoXs;U_RU!cw3Nrep-BQEvqWbNH zLZlq&HzYuxNwPfnVT;6(tp}HqBOSmisl(r^SBk?ro@P2F!O!7+yZx%D_Y^ROq?c$1 zxX5c!+)0~ZeeDmV@}tP+c`&dnB0NsL?ELnS_T$w?CphW>{Pa)lW}{WJ}|@aS+r8&KH;SYgTe|CQ^~{tb9I* zmFZx9zMw=nR*$ojCY>+i^`he)vkcvjruy$2EO zPWAIT7p#|gzO9=*XJg5I|Haq6sYM1;%4{CE(jzA-;YeBh4Bs6_*n! zcP=v9f7f+mNkb;6pb9e5MgrQE&WX80g&*|R&{mvWhZrNpcJrejnWl<5u@h9%t_vi_ zsF!AeYI;9kjnP5%>dvl$TBY@*?-I3L?sD)A>DV;9f8 z3|JOdV;llZW=?(CjE4OVxB{SNCD||NF~c@c4Eya<{_n5TBZUtE&o2W~hB?7kG^NJ8 zr-Ad7+%~NrhA)*X?50)2k}Mw3XHZHkqjR^+NPIZ+r;CO=ejKwvltfIH0sO;n-9U(9Fp4U8ox zd8Xue`fI=w(h^37&r1a6cOOJbH2mJJ$GLpWd7-d(cj+5?J^(!%D45*gx-zi`Gm=Sz zUgeLf8U;VR0cK_IeJS$qyxuA2$0+ezSKo^vVr-Q#ZoXt2vbYvtcL+;?Z@kQzh|XQW zP)krx-$$WWeDiBB;nw~epV+xzpO@<(l_mj~04%a;g5~ZK(u>%R-z`xaa^S|oLZPYW z#kOCv$jNDU5@PMt@ZTdZLNn^ohwMsl^{-iRyEbM_2#e2iz{WdrtixrRW}I>r*fmN7 znF!!4^%M6N=mZ%z9(0Sby#HYB@s^jzqUD5ZSWr}9WleUBc%b#l$R!cPti+##0pLiK zjG@w#N7NJNJj5TRro;w{8g&CqBTv#xUwZ5I0ku)4E;^hObqS#~_lr;xgq~FIk-aRx z(}~Ex&mVMpiLBJ=q&byKGgpyHjvT|gRvSu_;!TMHTJPl`W|$q$#u6u~D83U_3NCZG zu{~k0`FYeE`HCL0Zz%}`zIj3TL~Jt{2<5e=$tfe$DO>eX*K#O@wLn;v`gFu zk^ZsRd@u4ZUnhOeWZ{1=prR7tQsQqouN%ySh(Qf$Wt+ zoYMGzCwGNp-DGI)Ell6KD7>XDQOyOQ&H(nhHVnOoQ|hq{4;dc1jNNbloh`zs#Lxa> zRg{$Usy3-=G|pEqi7}e*0#oWoo1+N49Z&oy#H^hbi@^QJy{-E054imlbq)OUT-BpE zJ|A&8hziq+)7H$O=QuQPxj?SLB&jw`Y_siTYR6TFlbCAu_5P^s`f<3>w0Yc!tW&vZ z+Lwx78BV0bWT6vFh5w-Q2lNzVF3WkKAVOSyAUl1_3=Jw?_!A?ka-R)crqxSQg6D8` zC0$I{EZmmhXn0}&H|w8C#NRrb19b>6L~VYOf)(8S{-CVb32?syQR;sD=bWt8OMuxK zM`u*yM1wXkvTO6En$2CNZ+8-7tDkorXlH4vIb+)At1ulln_HDBn(6y)KI%;#VhD_a2KNT06P`lG|!k3i<$@*GQitHxta+?O6f(x%RP=dKNp-T ziyrk+K+)-VSTR-Kx+nAnqH}-g35z(XkAkCm1VYRpJ$&SQfs9GjCj{CczfsLN@7|9Z zytRq63`+1xByrTn*Gf3^`3Z3u81C( z7xgi6Hr|cfjaq~B`#cu2iWa-iOq31^@|Lt&KK!7s@G%B;u<8Y!sgGR;rUdMxYmI%U zl_{m;v}@2AI@&!I|DnpWnUCwDmmp^IW;kbcl7HTn^(a*J+KK2yL*B>9dRK()=PigG zQ6a&v$|sG*Y&CUKi_B)TqPp!yn7J+^as8+=-~H>v zujeAq7qUxUfb*&5hb;7oFH{MxI$$1im8L&gF`4;no!J4De491aGf7a??8)+wq4@9l z-PZ>*`Ue-D+m=-;!QkblXMGZH9Ljhi3*BDaz~UdKU=eiL%cao@kapy7f8UBoV8EKk zYM`HtC+^i(a6cJZwgtzFbGmfudTF-fQa2`K-xI2?~ffx=n1s?{X!Bm_zE#a)4l(q?XOS1lm(n|p9sl{oEm$ z6b%%&#%jb0LRZI-!;pZ#rPIkPq`yr^S`zfK5(9()VEE? z6Y+1lBc`oeba?38$jFMYERb7&o*((DVQN+!bWsQpUcO*U$vH)W$k=+``n@L9hy zUXs3~d6~d?*G+IZAaSQo&%=w#L)h;bWl0E!@!F%K3!A_q*m$i)6!oP?rDLSu96N62 zb-Y+(egdP?+A1){r|ekDX1Dz28y|0l)I%U##_PhhvaI*rDBR3#qrmHp6E$;?-!Z84 zlnl~lwruu8;SEuJ9}Y#5&-$k%E8DM=R9)Y%O2X>|?|>irO#VJtYrXmOitZ@x6^F|R zlMhOEXt@q}p@z42ZS__z-TFyz-D@1SmUpff6N8%PK)*hoPc(G~QcK@QLm6CD$0b5w ziD)L+s&x3befh-`7+ZeBH-=MP?j^lW3aCN-v*6W%5;tYF+#e8322dpPLBiC#zvjqw z+R-Vs+g^`OwIrZ#0L4PyTv(hmdTPn|oS4(gj(0p03MaNzOA3EYyw;^O=GKfa{Ia)c zsCy&};MLwB%2`~Snp>yzezVQ^e-xX1Um?@fllAoRXUUsvNlfYTV>T>uJT^n!1xo23 zc&;B_YdOe(nL9z)kqHsK7W%dKNAY*0Jyai!$~gr#jD1;Pu`i*K-+R_r+y2p@_}7F+ zmnT6T~d~H!S6${^EKPd1{6(r~6XF!ItPgf;&qPiNQ?4`;qI5#%Mc&Pz#e8`xNsR?JQrD)4~0Ef*AcJgn$2S{ zh<(4kK&_&**Wn3ir77YgaxvdCV)BY%vBe#A1WI~pPL!L{6H|l&A`ijBugaqxj}pKT0;HS* zU=AsrVvFPLg!pEICzyxo7s~u_EU~w~8P2pF9Y5my?*B-Oc}-zE^3uuxlDeOF7PrL3 zTE0^0ut}(k^)k(@B(QB+_6_4_U#8UK0$-=1c`0y_j6GM`>Z&8qTk1Z(ea-8V?%IS4 zDk&)&`Ff@`bjucAWokuu5AvRRsZOjMoz^XBzQ+&SO&L0>!hgw=rsOD<`YaOz9moWj?S;SSISsX?J%`e)WXKfXGW8O1QMg*AI0 z_B@$8>3!};qm876W-7mO?@6(%rExhOix;AH3DWD|suWw~9NyOO7a5p~?mx&(z#`y_ zZ_z2)Wv_yM3z(m#_G|L5;ZkSVfu^6^7Au}@5z~c}7=rKsGdd?(AI_TL!+L@l)}!KK zM?eK-qfk*vZ|n&#-4ec~aL)pOlPpJ)n(Qx}i)cye=3y1e?_Pn`^)v>F9{rV&?%J_# zT`I}j@m#wTOhNA4Fg70IhePtBY-z++XD*`6gSwbek%rLuee!LihDN^bx~F@T)P#kN zuE}&^)y6$WHe0DkMjmt9s93x6?Qyo0^e8eKZ4QrzK=!7jX2+?7V<2sY?85)XX~~Us|!9+ zi?{4!OTmO*MB@xSx*yR>Q~;#Cx*PK zCi@$Km``0*U@ERP-#+=vY;_sv4)isfBl6}QqgcSkJI96c$ZLN3v<^p&e<{J|FNGVj zegUyId_qPt74Eq*->()MUaj>%Xd0J%K2RWn5p^>A>{A~PbyZW07n$-cgf%v6V-X;@ zg>%`^7zH*FIOg_?bO~h zn76^r9Xq#jNB!)XO?Ee1iWRJvR-u3pju>O~1qM^|h>=%_%JL0&Y-L6W9d$ID=9GfZ z)VzRp7+LJ<-4s$E^};GmU&FaOE^&n2E9+l7aME6Vjfyvw-XjN9F8ku7LK4dc^X!C^ zH{?&%DPXPt>Z|E`&f#W@i%`i^S5YVBl_mfUtgf~rebBgZBtrECvHx$v^33w(l%iG7 z=L7A%X0e_Xz;Q*B2vt*A7f{92+8h>*K$$5Upa%@xP~M|0cBd7)Ew|Fkj*|Ul2WyM-Q*sv`nMLKbxhPzi9(N89=_IqWzL_j}2CeL` zhG6aPN;T6O4_lEy4<&v`=g@pc*BF%}~pBQBVQ zGWjrOHqq^9bKEG7RepUk_eqKdl6n?bJ3Aw*RcW?rTpiTk=_MSWlHvIy(5-v{vnO}^ z`14Hfy!Ss~HSP4tMHr`wh3=R4pN0t+rb%l_WMrQV;nR?afon&?l-l>>!I|!_vRArF znPZblI9D7exT`q^Cc-z->@NVXkP8|R7&|oqxio)CBI17m%WY`K8~ck~a@bkzO5Xc) zm~}Dx*^c7h(EYp*7D^fu!B$;RlqPu>xsZpKzx=FR;NAUGB&mK+W z`YwcHyNbVQZ(#J-p5!lNLD^pR0qO2RwZLm@$r@Fjl4y_BHsf1&)sNh{GoO9mdgD~c z4cjU{@GWnQp+5J8lqwi-T4w2F!e8tsQeBPOD&{=iR{}X7%6sqVk@3Hk_C|L})FnBC zK@zo?z0gwLL#0U9Ogv_f#ddbEISW)aXXl1vUH%3mmc;_|K<0|pSV0o|MCn&Qb`#Q6 z^X+vU6MbM>ED&JD=lLxzj!~zHi?n}XnbM0E!KyWqsrzl3FZH3N8qL;-Fbzt%zwZYs zPrk!4JR+ZX>4%Dj8mCU@qzhR&rO(+U{%Ue-3SPtlTS?YSNi6Ka=t6#<)g!{;W&IcyPTs_(=!a*X&4rfTMWBFmh1qwYu5r_t7( zqp=^`Le0Jsoxcp5o<5<(q<10KjKM%2k97}~Ji(W#SK-D((eF%LEOxR9$1t;LQV(`^TVqCM z>&8epNM7?3vW7*(w~#5kNB_}znaP;Ig6GBbgNL?(kS)ynL?8QA!>&XAm;Aq#g-54m z{HNZ9n=r0HLs83mVU49Q(>(OOGFO$NIZMj4){QyCB&eJl2XBq=dMg@#gb{0{g9Zvx zZopm!)FzgRGOV}E_d3*cgcIG+M(NX)^me^B>W^nW9E|3-yVqqNywTHhhQ-eunwvYE zSOIDu85vtXSsNDTNZPE!u-aZ1gj$zM8pZ-Q68pAPSF-)e@S z_Dyh>R7I9W>kfjZ!Bs3Qz3TRKiD@F^Fb(`M>pA0!;BRR z?_ADk?0GT?n5BVj`Rc`}UY!V#=BY~ii4%i@fL)WM!RCs;v3wUC&$1zg21-MJnVCll7O#Ob#l3&ycglQ+=(k9xk420o;JXbf z8F7^gCn9;vh=s|}gYzkMk1M69=9vky0cOEV$9v^YN8j&tS3y&n0Qc;1kWsfVuanB4 zvY2GxYFS^iw2brC}zFJy12&^3113 zPK!+5E262SRB|}&{of0Bfc5t3Q#0Aub*4`Z zH0^#i!*9IGZ)~Z^K6O*`qOWe8fUNiZa9~8dZFyhqer^C#eN;|8n! zNZ3=WhlTm;yFMJA|G3G9MZ#5Q+%%-d4spPw!!w_u<+pVzzcbW#;O;B5J>DWF8fTc< z5i}L~EY!zj+H*~@s_{}S$-@dD?d_6>WOz>UMD5)RL{y!ODkS8Vjjm45ADnw%zX-JZ z@%XusPYsl?;-)b;2TAFlLEY;tx~xY{>Z%k2{FJ998CRQ#lUq!8;`RYjm6!A}Dt@=R>^mb7G4 zr@VZp#j~4t`yOOSF1o?Qx)^DQLWqbjqX15e?@~*n?SVypf-0e*J zGY}ul@WSaCl)Byv+`AQjMy7rjV&ewKpe+y<10q@{0-O`g?nob;n8o)m7JSMXA{H*_6bz2~53CC|M znx3h^8$bl_6P!D^PEkq&bn)+JqI69Q_W;w#{q$9nVEhk{df0H%o`%7N;+OPv(CWaV?fXwyJC@Z~Fjd35!;?URG#a2@PiLXecrHH(;((DUKtrZU6{Tf} zbz3vi)8dfP0&98mj=nP&)#FHqt?tKMc2MxXC*6dgzY1!yoamF<^eUxMvp;tER^Jp2 zvQVDBYjK}+5|ob`UxaL0g0~HhUWenl2SpZhNE2$7&(Im;)*Wgltr)P!s_C3c!wjpe zNjZDzoPG84`Tl745z-Cxe4R(q2Q?8Tcy*p<}USLN-wh7Cm(mXpHq$T5KTL#!pIUxzU?mbVh z65h>dGA988SQez7L(lpH(&~GGI`EH1ol{~`zN5@MXD%zT`ogd6g`(|~IkIMuFkV5M z|EZ(B5?2UtP=vvxIXx<{j9?_#3Yk9)zV2f2H#TT9HBY_vSu<TYH zpWvDKsyS;A^)M<21nt@`1IW|D=v>(Trx+iWdMNN{XrfmcF7Tn)osfCPk1akB+<$Bf z7Pa@-o)L1zA)Hu-s0W=u9!&?Uiw{(sELyuI`~cNYg_>FLlV6>!F*Q&IBg~iq2{3;g z=(Kr)WXKOR?HJx`#c-x#wjk}iYMQ<&dlJt%I;}>hBDyRM`wYZJ(<8imJK#PZY6q=b zdN!wJKXk3YAN}NEez_9z34FLp(B=Mt2dH7@Ap+H-e_jjQ2(GN)b| zmEMrLN!RtVTkEx6tS#~QT2+;P3nLy_kNm>Xlx9$qmK$?GrUvf${R8&2X{4{JRo?fY zqAy;hot|&N9AA`H5|@r!aSud5_EteArg>{FQ{M?`_EQB*$zkOUY$GHaj&=Xul7DPs zaObQ8m=^mfV|VFi?9a01e~8`;A&nGJjCD0(gE<=oUCFU1K5V|I>O^tfF+nejmeoEqYg%qDN{qD}iz02}T1nXQV%*POr0^OUd$1@gsa1mfBXf zetNCa)p^v~skm}|E?Wui^LVx@$G~R%3qfxBfa+Etl(3puZiL70nQEhP=&{2pZIYGkLd&h7v z27ztj8}7$TY2}#PUjAu>U1B)>)_u%}?dkf`H_yyEW}=6(yq&+rp}$(}b8cTc4Qx{b zL+)M5l#JLWND(OvoAIk83Tbhwju!rGcuJdm76+m z$FBO@A4NHqlk7Cp+`5`l_&r4`t8eeQsr^n(c3{OKiQwL}A%wkRsfw=njk04|+{(y| zw6*-noil{`>GGP^fWX>JZ6U{27uo!YFm>^U=L9e^lQJT)&sK{vRV=CQ_;<1Nt)X#(tEawR&aSo zNcy)2dXEly&HAURlj|v==0J11+s?WJJ!rL}fd?i()#M1O zTLUbYn>Xh5iMFsopyLw4g9f$+klGV-rSMLE0n;9|b&nMbGGP_eXQ2)q5$J*?t=?^m zq*@ZC{Z(=M^LXtXP;unzoZ|Ijhv&*+(MbxX-Ma^GrtY!b(ge!b28A^}VM|*qkgp}> z()gBl^|)dkWaF&xpO@C6tDH{i@W7(cM4Hfak~7CD0kG)wd_oqJifCKs@2&{27G_=W z0c+}usmeOQ&4!!WgWg^D)r;fB&6;woQUZk2rsXIM*%LSEG|_fKkU2m`43&+Etmibx z9{Os+OMM|)?Xs0{p0ewln}>^9Wd&oCcb&Rop4ACNd@BtGs>GN`XVOC-OEHy@q~#uZ za*Xd$+4;FlERYQqQYKRsMLP;t&at8E#G+| z%ONK#)35enz3gd+C+F1(iGFb#%H9I}CFE0zx+#K~iUo-}q@i>B}vWcTz>IdPOdDVoxb#33Xjf64w?CR3hIL4=!Vu$n1xS zijh1#=53vhD_bR=V5@0|w4XljA7Oh@edPou=q~IZKh+?1WYZQ)jxLR*=#~ys8U$vE?pHRH>Hb859ZhQoyZsEnjiqIGcf^swCNer-pb;{KgDsJz$x7ugpy zc727dZPV4uqRT(ZCCkSR4rbPfh;1)8ldY8PrZupS)Lon&+-3w+0Va%zD$B{*LF>_$ z11Fjn9pe>o+h)G?OJwbxiF|EyuI^p-v{9tO$Eg=r>C@ zd#W%syX})~0b&gnz#c!>4xoNuHtF|S>UXN}m)-2)s0;=7ky2{XhHGW?!AIY+ZGDHR z0GE?~&rqzfuX8^==skhpW|y}5V+q3Sls5F8LmnlPkT>~DB&H>4D583RbY1KC6za&%2S31v#0-0a7 z`PZN+PM#f}^5uJTXqN#Ef@Zw@J^%PXuH~lT4`HsUFA+*pimvNkIt2Vkd1I*)d)4}F z`O!^MX=gA(f%f1ECd{LYXVSY|*r8r+P2Moh&jmlAGN6$nTJDkx(H1VIwPmdj{m-NO z!<^b==l)V+gL4%2I^9^TddN7bU?x15rQo4mal@`4kOAV1+DO-hm?OAzEZIg%vLI%Oxo~s%Y#{=n{g6c+2 zzr0MC7^+?Rw!IZ8iRg#z@z1o|gX;YBo$FY2_zdfKEYLMAgJs4cZ_l2-gbjT&N7U*B z_zdHx$wXWfvbPP+Lec(!qgu9H`f4IpUT!#r73}cUTh%_^4FrFlslpZBnL6yz8GpIEx`$!_T!fKp}yF2^-k@Dt$#ws ztzaPeLFy3x+q_}N9jbS~ei=pZ*V>^@ppUd>#b2dPz+e3m?9>iRZ+vv4PHEusGWa3U z+GRE8jLao1cL_j9Qmc9sz=n(0ru^X_U8iVvOYFR{&eCNi{%@CwwPVH!|9lQxFfagl zeDJI$x^*YrNL|6S##=lU9teP#@XocOJ|09c4Vpf-2wY5)3mRYyG&E;ae}tK ze_?u!g=N5lQ;r+XmuFL_dwD26ve~n>AyCzBU8;xn-wP?<>#E^O{2Z^uPTD-T1L>6D zjiaCa60BbzL?2PgD}SC;&XI+XoEC{Y>wMux6_e zy%A6NVR%M^#P|p1qiS1b0O_S&*4sD!`>dx@dqPX#^{KMO(`Klb&ZNeqE=7Qfw5igM z-5iV)6?I0k$Z>bz(!S&pZ=!$57ocP+V%(4O`MLj*XoD*ugEPy59&twPH(Q)mWGO8Yzvh3Uh^)_O+7q`k+*GuhxgZ{@Z^q zi1^W6K@ms6hYY>XE2#;3^tWE6Jo?pbQH5%5#>~oKI$7^LW3sziU28Bor{i-T zwVbM`36MD8;gInK4U(5@OXPrqGvIwbMYH)}Vl)~&bXRLV7TBWDCvicx@LqRPPg5M| znkzEKS(CnNt+e~wVw-EdSxgr%%N z$V$5pcB$`x7QU7-vf%$ay~KT9f8xP|f1eN))r%{Btl)53&*{&x;|~r$d+Wg4EAZuw zMoEk(YW2tQ#S=f2g_d?cHb9pXomc&2oo4gd$iC8DD?o8VI?Z=5D}Zb+_O8NHtDUgw z*4uS~zh=;|c4)Fmf;In=$s>xw^Ll1ESz%g|aXL?^fx}-F{#6X2ZmsjEu7x+}Ek69} z8%ul>^sDfvVI>OV(|x5m{<>zKsmX#}_fM1?NLgRRFIa1_0q3kz`rrLNy-avJAh2Pj z!{=33y|McS_H~%@-UU#tfhR^!lg}QkOWA{>$iTt5w!u;Rz?X3yi@S@%o{NokS@%7S z75NwF=Uc}H?gD0q#jtC%bK|m{LUoY7TT3bAW@0xYOnj!wAs4>d&mDoZ-nE0)gv@uBuk`o z!Fm)7Ak{W9Q0mW@Mw#&%W-%*3%|iuNheRVLN2x1UD6ECJrK?_7ep>oT<^PdZP*G_* zSOf!my@Gy{ey@s1RAUF;Y3@u=NEmN)azD!g94M;5u;pD}ikKlzfQ~IbEDv8#VHsw7 z^YFch^8^osq{Ox&DYWZMCnH7_xYWlqGz&D-;`}+2@{S2snB^TkS&M2V!T%QEg+BQf zMX_AaEyF(+Kl$j2`7;E)_HfQ_3PnI4@}X?>K|$jcK--?gefUGCt^E(sGifeqRn(~W z_N&b$`vHylx4V&-yWZRt8Cx6*XSqE*bZM{XT*LdaRJpz@S;yDXt4nY0o(PXW5s?V- zw{DSmj1aHVX=8re@^jCtz#l*hTVs6*NEJ-2Lj(v5{qLR>$P{+w8_R;*Pc@xMI$gKC zLzMlvz~ND?ni{=NwI~XXDAaeF0gAU=4}^bG4+;VHX7%7DSfpA~AoR7~$A&dTitzX) zmB?%R96l~~`sgQj;-?rF+dgv$jGP>oeuFgL?@zRqCTLy5P0gNP*g?5=^Q^0Pg4nw9Vq;{eIb%uw@qLkm!1HUdh_83hjeFdn`nc|Jc}U5r2L zGBZp6PK=ggW$pFix@dO^kO&V3K6|oPQGV5Q_7mMSondG2d%muq;h)ka94wn@kxqSg z9N-bx6zVR95J?x)5#Hc`p~v6XHmmd36!8QY)ndUZ={pG>xkz~kfCrR&fAQSrif!MW zCIA%(1Vdr|pAbD${r6bP2j|rN4iG?qo~Q|;XJ^44CL26<8AAXsFiKC1n^G`6hxSx$uN~DS;-G8YiMGCGJ1-?OAstlfIyWkDX4cbpB1s zin%rF6d81XLh2XS1vrT+c^3SqZo6{^u!}-t3|fK|EhvjMK*6u-V7zg|SEVfb%;sgT zrhjdw&iGvParCZ}&LkvUTdd4KV|eXRXa!%2chj|#X{NJD+1GwQPs}BC_(@yqD?$P0 zW%?b0*1B#blN15?D}-5T`v4(BZwVR+h(&xA_?H+F1K{GH`ckjQxO6#KYj*HGAYCFW z&UJl9s2(|(bRLM0Q$x**f1jYo8ZZC9tvoc^#RvkN6$A#Cg!v-FyVKnz-dnHd1}-bl zjND0A#QD|0=bsvEE2u5}$Y2L!YbvHx*;4?`oTrsPGG1Ht2QCBXem0LoFa6q{-}Zwc zTa^x`2Y(wVb7k@Ts&k*~PEiR;zrBd(*7N~gLRNBY4gKh->>B=6zj=KVpC6O6L-1W9 z=tk?BRr0rZ{CrOqK<1Rfhi)%?6l}0%3GvoR-JHdDywuD7<5fq)JnjzQ9;%1yps?YL zf3Gj=LUs;~GTpcKnUx#wudCN1X(fPtMyrBUbEyh<27lK|?8xnynd(0$WxfNwWp9iu zp}}l_y~uL9j(%zE#W^WI=e3g}0J#o`eWl3RD)ObVRh1bdTGeX{ zoCij(-`*S3S_$!6d8ub6wBhgGvEk$#dl zJ1PVU2Td6am!uBBJ_h$i%O7Gx?~5h4@gLqzYCC}+dg>KrkQm*&5OO}jpyP*UY3%ET z$v@6-6Qf53av!JGE;|_C<$o3(o@Fs2V;R&pC)*$&#`np2z7k&SY;DJE?_PP#33Kg-V4!^!LIAfU&az<15qy1Q4#R}7o{q^@e90SMP0JN2c zx(-kiFszxw6e;<{<7+m~MJz3^RxZ>Ye+`Ids82pT5{R)u4Pm1t!~uz=(uY_;{=ftL zk_Kz|=ALwG{-*j!$I@DLgp~?YBmRt4xSBZp?3^WYgV@tV&raX{*r@!TS9?Hs1F%BS z-tC`r12^kVdDeRObPezj6AN-7WeGIcoaCpqJC|g*;X4=r*FMpT-1wpn*BV+Mt3Gu> z-cd{-5u#2*??f}VZt4>`r&1STC7r*ae_2HNz9-{?tZ8*NLkHp;Lkm(BdMi|Q6W9Ef z6e1nvrk!v&7juu25x7sVl=9N#W=Mz-nDJi58;wmHw1{`N@oFcs5C>(&h zG??`>%$^}j$?0Q^%|j0XxvjDo^utkC)EJTd_3P%7wScP|wPV14(t_ozb(&ZL^_D&= zv^PIjl}4ggjVt9b;EqVxNVrsi&nT@BX;AgBy|U+dZNZ;%k6$?4X8qVPM8@fpcO(~} zp@{^q_U``N1Z1Qxs3#PY=BwR=Ax<`bpBVMAXr9wAHFnBzRcGe7zhiLMd(BO8Ol};* zka?_9{BzmdyOz}v$(J>j;q(WWKf&|*UuZ-?e=P*=EkN;n-zNU53|-Js?4Z;2IE_flfF zBj}fAekL#ni(N{ypbsk$2cxh<4oyYEkZkOoqA+#pK!33GPEWJ+7MMW@*TiX^-$~M>I3Kd? zyBhW(@v+#iA$to5L5vE%S)trxedYJM=;JN9y`bVMY2-Qpz(&0U5yoF_JXAV-&MK|L zH=H~Mgy$)?5i5JbffHGeD=GxkP)I2yXgJ3=z()CBvxvs;Qix7H?_I*vk{=%0#<-L~ zUe58u(&_q`9{RlxvY=>+R6E4auM7EG*DD16;oq5_^96H*zlve#^8=M`V3>?$jJ$rv zxJA{C&-QtGx@8i9B8P(!hHW`irONaG?`AA@i7wdRTF|-YZHuJ2FN)AoV&ItSi>xK8 z6}pMv?pM*RrHW2nDMTC`4w)`0FAP{3*KI!u_~cyZ0t z!5^v96~PS!WUiu50cg5M^Nt&Pk}S$9*ZiDdeAa$a>$?>C+TdXZxA7d=wm^`Pyc2*iFKxOtDF)^8z@3LGr-|b* z;bxW52B^zcKoq0>i9yefz{Hl@M{Vnb-;EepG`UyL<$VgtQ;gngi3`jQ8@@kQ*7b>w z<1CHIOK@6V+purKQv4kthFSUAm~JXkwl3Qtc)kFHHxpQJz|atBU$GzQrP!_N;&MR${+ zqpqGxo;F;m#!SZd6v0FH#BQl8M*PtTzOit*J>2>PZDNyOoe=sy#|x(T+&4%WrxPdJ zW}_{=@WK7=uMF**uJf>7{tNWO?-a~q>MK*yxu^M`wygA6Ghc@x!=D|WfXaUB)~>28 z;*uVzpSEn)TmHZS8)l`V&&Dk+K<_j;y)N>R8TkBOcvF2n)C~dXr`Y*4O>Sv^+n&$h z#TI=mgJfqd>8BGWIaB~NvdfN9ltF=N0Y{aJ;ru70 zY%Y|AI`koPd*rIaVtPo2<8X{_=A`)GVz#ei3Hn!NEjJ7}3UtCIBw&gC6crlSAi#pp z{K#|ZnZqs&Aj&iuSi@mo1948Q^3D!MS&Jh2bg-Yy7@e%qg8`jETMy74E=!*UJRqlk zzkeSfSgJ4^gv$lL_4|A$V5Qoigz<91%Io(t0JYp#M93anT9Rz~;@F+HzU`TbfgEeNKm2-tpt%0i z560LUOy=!|5x&u~05|6ziYq=DpY7`q|MAz7>}v`K>XxT&j3@lyj5Yu}Wt2EMJ62Xn z7g4Eyx`W7|R%_3-tG$LmQo|hiVHw7r!stfUyi;fg8x{?W&va6hsh!O}6ZdKHt@pb> zGiL)jh67j5-;=P{F7QldM)I{3Q9n)wf{TOX@;~R&gV;T}zD?72D%6_g9h+_!ci&up z8X^gIv97+xa+oIzY7N{rT-uZOB+D*_HBkLR7D5gu8A~(Jl7}3ViIT>Z_thorM8%9VTv=1Tfz?ol0vqp>Gjt{w}>!hdvtRl0O16hI=kl$dyOo(h0Q-@84w4*mjz79F zzT;K7{8mgx7JaR^L4^~@IT23J?7G}2nw@i3dVS-q$lR`DbMXUsjboYp(({t2u`=D% zs5S!wTGQPO|D>!vG1+g?()R%9Bjj$c+JtrYs)eQXbInJJ$7?wQ&I)O?G(*F#T^qwZ zfh;z-z&aaon3CwWSkRFDexq6WlB}L zFtNFKT}?fGvY7o`r>~A_!}DVS%>H9<=rQ6;pOPXC2pHa5E%sWD%GXQZN4*56FmO)1 z9M0T)7i-T7WdZCO33ngi%5TJTk5@$rha;UhcB548aZ=sSYw*B)X(%A*EhE=xgKZUDO&w2+9o{7& z)5wf9sO-j=N!ZHNT6B4CH>;kP%G!4J*^_WUmB}fN@ArC#0i#d@`hyTwA)_$ch*?_k z7|%)b)AFBy+A1p5B^S5W_T*<|u|aOpUCMu+$FN*dV-eN(iGxxw^T~5eSej$_rX7vJ zC-C#YXOsG~V+J|QD>%`dFJ#GlntE%RW}m)}vXmoTBTE0hOkeB9mzDnwr#hGuFX~zVaRUC$yuS73@i}J4ZVxVi=Z)tQQsHP3|n)%YO0moM` z!T&CX$@sNKTkk!C(DCKAC~DRnbGA;JJmV9WOmXF-q4%AJRd(VgZ1PY?iA*|@48qGS z*v8EgW6Rn&=qr$s_#}$pfd{TNgDWbFP-OGh^~G zzjl)WtG6u}=n~k?98BceP^mVLVhuU&cS**jA#KH8>fz&^PC!_QGoip%Pp(KBiKTLDk>)T_N zj}ZR^6Cahb1%S7R(O-oQ*m=p~)!%dkJZK5}riwI@)*!`YBA zW{#P$^C|+)jwOAZ%(B+==K7WT_xA<@ceh{pi;t}v^hciExjpyDDmmRPgsHQ$eQTUQ zE1*NP8Rrk^acuP4+n#dybeMnDp|VyC>&kOcMos#v!)>Kd!ZxXAN})hn`l7$l-pp&n;qB`9qJ1R3TzCpuneOD2oEEi|C~pj>Ac}7nsJ>hsSfX+?Pz{I>e^H#CcOre zAxGRgN}(L_t+SjTN90_`yzi?xk~Ia?5efsfu>?RO1CS(0lG}Oq#j`UIsFIjDxi z4qkTE=2xrBI3m4)TD;S&_^w{#3J9w4()8d6f$=`1SrcQ|<2TVAAb}mF*pcsj=dgU& zND1jVO8M?Vv(#)?r@84lIp~*1TH}1jM%Tx8JkdC2ffzBOZBun_ijF7zRaT(8<#K`N z$!Cr?3cGZ1{A#udrDRCOV)MKt?V4a$HH_TJ@b?C|c@Gc>`(0f*Hg#X)|C5s4a1lc{ z-_;`-ZVCMk6`~ZFc3G+jL*1(<2Ss3U-JRt8R@)|)EWv}2V;cUE<=famPmxb+KgBkF zy?gMtEkN(U7j)!xp`y>!)}4_*MjdM4T&KjfGVtfA>IJ_o)io)P>2=^RjyENeTGkZ9slnkS4O zP2;Shb$M5GEk^(*d1BW|sL_;X4M;u!)LNQ=$+6MGu3119;qFfH&R2k`RlPCZh|O;d zxZT?fq_c3P9kn5XJ=?GlmPq{b^H0d(x9AeVN#C^Q^?8;Y5+@VkVoH-ifX?{%lPr+b z?em>175Fv910>zk zDC)>QCw=Gi(@~K@eXZ@ERXJ&(HWq!GEVF9R@jP|n1;G9+?{EtTe>ZfwIrBzul0bUa zeoyNtUlpxAEg4Oyj527j+?*5lRJ;%1>ymI-+4Ke(zK+M1K$f^!S_)j$?k{qG4&48I zDmJ$}kuHQyS#ZWL6t-OHBly9OrzM0}ckwTv7@DPAV58eWV3Se+F?%&w72kH0RN#wM zdlO$0MP7KJXn$gb@~9FnM?WmBn!YQeA6@-!@789yQY>%XbyK)Fm3?QTwuE zs9?nr(99)%eahWDMn|{v*!U`h!2K@)+of%lg?Ec40bLNM=Em7vV}-Mh0t#OHjh-tsT>nve_b@k6HnuHE>D$ zw-)6V1iM^b|4l%2e*Wom4wyg07TuUvzt}EEDiO`K*Mzd`)8vCoI^H1<0`GHXq}BIV zB$V`KUbH)+7plcA`D^wqPJc;%h_#m@>*Mb|m~LmLuS|~&DhHGLo^$ zN^w_6Q|b%uB>L4R#Tl!aBcb6@(?EI0RsTwnv-9~Wf3}$#&UKxUc5@9jQ4KzH2o?{C zHqhwg7Hlk!0x{9~)q7y;73$Uxf)7p0F1{rHT#jTF@ZkRWj&DjE^*4VTuJU@7b07pP zT4|@)7qxGsqDL7~{39e?oge*zz(O6!niLW})(2>S<$JdmJ#~u&-Om|x+y`PQAG2Ou z>Wj6;mw57IoIw*%P_B_DubNc&qSha7ziE_H0Ho-p#kU?(h=nutO($^}*NL`VbfC%y zt?Y0ObUf)oU$?Q=D(abWwy3w;g^nyU3s@Kc%8d?_WqG4p6)J0Qqag$2?~=-odxv;; ze!;ecrC~B!;@j@i$5|)6uMOR>N)@9C8F1y%p9uey`EW(%u&7ECXH0Si}39DAs#g|*9hON6QpqY zS4OhEBL5a(e`GcRO#7=fV020av!kF3jR{S72w$z(b0o5Xw+qwOoxxUL&aSl)-*5m5++>vEv>5*+uVcRW8Gb|`Zzs*MhRB#LK zX4jMtS?+b9h1K^|CG#%;(v0(bC>hM1eo~av>Wx|qkzulzz2e>zmc^|W!dGT#9?i)( zT8k_x3@O~nVy-v@R^t7=ENsj|_TIf=YtLAm;p|@i*f^=#DEtD3;I5{mLN4o9qzuKB5^6SsewENq(XFD4hp{ zG)%Fn_ntlJqtsYstb)vf~;z>=4b>LfrNFI9FZOL*ZJzRTYZsk zQICNFPD|8q!z#wV(^|`}J8<%5vU!RSZ&E*M@NGg{w=F(3?i6`5QR0Z$Kc&!9KQ?Wp zPpx{KajJ@EkrV)kMnrpN2GrbD1qu6mFkT-Z zGxi-};p5GaBk3MRJ7+4nY(x);w^}s(7B@XXK^7SMdS*X(Sd^KYV%=3B8vmquHqn6w z)?I&Nqj%4!*uGYAz;*q3*0Rh%2aUl1NGjjYpUC==qqRj*i(;_+aF|;?DG?yEFLP+P zC*0Tuz{Wrmh~Bai20K``xI&$Lc^R*!ovZ`tL!g~udzuq;40V;MrCG~yqaQ%A9`$Sp zHJR3~POtIVtM~$l<)%f3+!#yLKb!$%K`d#$Lqn2Kyclk5xL8-3j5xtK|EXh!+{=aG z7^xM`9TLxGODT$D9E@1jel?29)RMj~{7EjJ-(8BBZElPwU1N#BzZCJhp*8!4kcnsV z2>KDFH=I{A{j$RNOMFc{$`!~??i3ak%So7YFnp5H8hx^yf^c-C39xm&{=vAk-h!Je z1TY#(Sh`x;#Fwe+?KIY~W9}Axf@ePjZxVz;z4&_fLZQzb9sK{C_wo+><26e=lmEaD zakALZtK$pdD-y8H>X^Jv){bGIOMzJmTGYePa8qT?nmj1Of zPW1plWSN^YXD)v|p*sP@-tXGw@3k$V`G+f{8-!85WMO z@k(RqFe%V=$x1;G>W-p(pdOupUVAl5hwRmamu`#?1COZd;#}v-hQ>V4!5*4HBFEky{cgALDj{LCNv+Jq33bzAKD-%Ywd?cyH6 z;QQO_T^y{wa`{V<6iQd+nU(7R(VW8D0YdD_<_Nn2!SGv}5g}b{g`S-*#`O(m)crZz zZJeIHRMo0@P*~G=f1_w=vTu=b%I~=tN7BOR3?Kr!FN&T+3GdDs%d5k6S3!zSZ$pXT z;%F(+1?YCYEl49}kkmc}IIkm)FjTjS60zKurWo4BUVlXi$iH&dzWF_F^ScFEpei0s+B% zg}O@R+Ml2yKLB$If<15kbew`cvcAeZIU_#5~ zpi*(^FH4l%EFsn`iGZP%Z2+z9+<0&c^3qKK;$}~q3VeWxew_;vZ3Ez zY2^SNWq|17KL3ulp!|Grw8&4qk6ggz*Tpe@=V)_m=uev6^4&-{G&tv2E!Y!kxd5i8 zX^RLpi?(4#>`vZM^y+;pD!}P+m98_&q`cGNuDCzV)YQ`t%5k(Gl*Fuk@< z2DHu^%R;;Zd#Q`hC!w^~-iuVrS6L(-!k{=!o3B!=G3Y}kWGz!le{kj~oflqLKpz)k znnrAiuxJ5tL)bgDdM}5Q1DzU5a*aDP4EjUjYW0;cn((`Wt~=kexWV?it0AbavXIw3 zC9*@^eldQsB=LY7*?cC-$9K2rDxywwLl>v$SpPnXrr+i5rt3iw0zW2o%N&O_dj zAZ79Nj;tz!wD&wT1=NPs&z!w}U<5JidVaJ5tDqF0F5zNgAkKl7QBSLqZKT~KH$`>O zHf${eRG`O18o^fJ28&w9?>8n?OSci}VBOm4Yhl~q9%J18Twl!ucW<_&UP2$ zv$$e8fhF^E;`L@k+NWOvCH}I}!ov>83*Hu}?cW;0kG548F1wVNW!0o=I&5YR-_YI= z{DErk4!ykV>Y>?u!yAeB7uXgD>OUhAiwn%=ay&Y-qZTr-H{s>j+!U25{q~qRW4PwrX`lmDF=r32}ZK4jF<8)=?KTisGU~enVD|(PM4C#h#Ru(T^2D5 zV|1WHY|_9sCyR@C5Z&6fKzjgJ0@ifvWE>Ro%b3Nq9F>-)5(1Mlb`nizvk57&rJ0uv zy^%E-9+(BktL>%-tn9v%?g3(QtG$`D>oIdHRWwS0=;u%b_kJ#)jWz+s?7xIKf5J6?oztxM31)rV&SG6kqu+OkP`(sC zdhJab|H3YX6J=$N6mN#>>?({nQU;dASefx12`R z;n|y6qv|4?yTo(T5n!N&RP0cF$r3W)9qJ>VJ!f#P5$RR;Q-R#@+3|{**UFCgj~soW z25J8EY4svjvTdH{G}EOH=a>-+C0H{@K>_UC81#%nXp~v49=5aT;$XUqXH`qTuoOpD z+mQ|?kU%V-D?SN(he=kbvUZy|y$Q1K+&L1`O;|#YuCrPt@ydS+2{P53j74c-q%6vM zy1w;uqFlpG4fOS%x-XYZo+;~jUx=g!61|UcLdHFKNU&$>Jl_|mz5<)!~0SPNiM(AD~1oMuEG4PAm;}o!37Z;-hLE zhleZd@U_$spK{tckeTI~W!+%LC)lv4tUDpV!L@duBH!kUjoQZVu0M(69|q|4kqYfW z{^BkaX@CUwtfq%V)0$6+u3hrI>kRli{=sDd>6r>lLvOP%N9_cGOQ8@jAD4qA*a8~0 zI#^(TquXDAHmjr80F)HcCqlNYV^MF-{a;)w=D@qiYe(Jb)Zye<+D;3Os(50J?st#}eJG-T=F!mNtbU~Nr8ao`JJJeWdlfZ1M^W}z#OH!(1 zAA)R0YELou>MS-iUNtLJ4LsSs0*};*`MaS5{=w^ry&l4!7EQz za#K1j=^}~%04C7RWDssay1BgaB3Hz8g(Mp2r4$qmA_W#(1yE)}#XbEwlV1xjn9PFp z_j>^Fo2h9u+jSvE=TH?iI7>fOT_^l`Sy}|LZuys)VmJ9eA(i?s_~fzwAN6wl7c}>O zR;IcNaIX(`cO&_ewQpQ{ar^pz{4j-6AshaPpPdVll(e)#{)(GY)64|o&rcM{t_FMGF7Ei@_z+kzus ztXfCj0-xJtDp+2gzn}bl!dy|7y-_aM2=uUo4IHBqvP1G5v`y-&)U91e2P#^w-A2z4UEBl9R^c~rS< z7}-*83tOCb!Q@ZZ zpV{iETp^9VBD2*UdMBXPntS>AoxfLk-;UfYpjkdD{`8c~-ua(&Mg@Y`~sa>$p#m0 zWX3qW>o(>)s!1VuZnZSW0DWL~$+puwRyTsdgmz<>z2lVBW_Towx-$2$ily_eRt|X6+TB*a+M_>4akFDdJ zw(hU&r?~p)rH-#@ypS^Ahm_Q)WhS6HqhYz{6ZpW41MMr%M(<_&(j3b@{5`jKCv(wB z=SOT#^9{83S3imiYkW3dJVJOjJO1U_ip#OWiba|TRC^SQ?4WUwM{hS%`Nx3#JkB&{poi@)!oG}>CpDymrt7GlW&Ky0<-atQk1{?{gyY5W*n}0fk5qqe^FM*w3cn*z>ER4+RB!_` zzM?$U2YP6UQdsyGpu5-8GPsSC08C1(S03iGOwck3>06^ARB!-=ezLdF%f((ZzaQP66?0TnNamLSP zf52)bj1`1iHh~i5S&U;Pw!R}fjN6nCP}doqSh|mw2EYcz$Gs=dLV1oI3%~u_C; z_$6IVNpxYhkyHCHJNZ>z$`kYn-tEgw@iw96|NOe~G-`pc5BQ{Ww91q1o$_L!;Ry~R9{fGqI&oO@tc=|9=40u- zg-_`Xl<6-W^X~Od17D~vo~ENfM?=Y=uXmAaf=l4{B!savA7g_dllyI;m0rhd)Bo5d z44!?0SvX}+)qs@H(_+jFAm)zVy~eoS()LzQV%66Pd}Zha`3tM%t)ruCUD&(N6=0C& zq|Nofdt`NkR`oix{gj+x`8^-ly^)hT_f%OcAw{jywFO5FuFXHyNeG}bHpP#Gg%rGx(4FystC zyF=8H9wc^b;a3auo6dmd#E;B#%y07?16p_o2DT@#jsu1@SdRoSUe_y+NMdQWc^BRR zOqqVspP9vocci(|G-u#BxOIVj2h^L7I%s@N9@fsCw)YLpyROUTRIh|V)r;t5NiH+f zcNOW@%~mu-pnpkappLjYP$I_Nw|6J}HtXiyI{Dt&qk_@A!TLC-Wwan}=W_8zxV{*Y zi9>B1C*N{lm*W?u3!uT7f1W2#en0TP{LTpVqmIq=ljlwCTDXrL>-+LwJkyAa8>F{% zV&xSHLdT8?27mv1-9MH)@{|>_f`@lPgmlvY@$w1L(TaQTuUb}GD55j+5&}$)A8Y^m zU;Y}>5JEp(D-#TG#-+cX0jA@>`n(k2$i$n??qk~=KC%F_FwF$)!T=4<)i=Nd-W1A~ z?0k{S^vEce{s^$az@JSze(W=NiY5Q2$3Q!F)TwgH=Z87hv12Q=|M$cH8%6^_zx-#I z^XsRv#3!5A7XG^s=;!YiKl4I?PD%eIJ_3GuO6)D<0s72M!M!2~+DN*dWvk=*=7Vl1&ar_t9hMTy$^c#vH$tUyV?+ zeEDSn=Oupf@2E4X^_>l4fMIjSnQdvFx}1!w!1I7<4>w97zzN7@b0DzNP6^&)LXgYV zGUOSNYAR$4FFnVgyQ)-puS4M8%=*4s{eFdBFAf@i@*Yj(EB@wln+Ik9n5YcZ9$zju z+g6=SIYkOR+}A=-7fbYP++oXY7^io84gBhY?-cX`C&;X~fC*$6ZD2U-q|voUrw+gT&$I)tsm2%(34OmfHSIgeFb9ND!L%L|(3gQu20hxO z*z!4SxoS$fAi<}BPkxo^AP0JLu!XBJHp^imK?alr>b^L4B+ibNfAA%8HNVEzupkN? zeF@L@k+T{Xs4hChh+C=1JyRsLEz@_VJU_R)<|$6sa+`6Jbg+Y!QX;-Ei43;BswD(A+xtiad2J3V%JZyXfARa1F!2ofWjnH-XDBTIWax9(SYCaE42e}H7} z8m!MWPo}a-V=O59yBXb(y&8|2@SQW9g`{xmB@&uzb{=PSuC9FC_ZbKE(=~ZKY+t{( z)A?{~(EoC^+Mo&*KNVX@EHLu5Uh+)PZ%l`4 zyWo~2| zgvmL;^0z3`k%vJOSNuUVc-9CrdSom$mK^lVhkkMjiCp|{Rn56yliGXyP0bIui_T8s z!Ct}shFXuiAj3Xl-u6U7h>;%+6kYcgS6VE+y0gyaO9|W2lz>pL?!hJ&8DB(clr@BT5dzJ6vtV%V_-dk-`I}~8EnMh*EY_7gor{>v#$w9T(B*F+z-dz!57jFJz z=*qb%54RKNubFY(9lse*4alPN?JmDjJbx|}TV7b})#b4pTb%g5(#IGn*DTX0NqI5u zMHdl~s&+7@Ts<0ouv#k;ZU=U!%oWmu!a2mQu+AAKdPXeQkSx>t_pOn9(y~<95sa@f zS8Y_J*?|R-1>hVbm^s=G?1m054JfHZi`@lrt%QXNZhr(sK({d!=&F{qgIp2hK^`l$ zQi4o-?IVe?%J$TEEiSX!X9s5`f6W~Mm+Alt7E338Xu=O9!%4=k+^bK5W89 zxA3KFUsTV*67C<0K?&NPasO5#yuw@?Tz&uoIBD&Tp9mk zJM&F#uHt6tDE}o7d%01@f9%ZWSt&-R*E$+PkYG385ub_BF^Y2?66#(caD?(3 zuxqQ4a_GL06GD=Qm4B$lkqbz>g29H(e|&>iN~`nt;B3S@zDEW{R)bX7ap9a?WfU*Gyv2HR3tU?$w$s%IRLBJz}Hbt6@^ao~hSDIW!`mK{m=& zdhgG|W;ZP8ISjDqQ+GpH4?a7M9qL^*@PiIa2Yr5b3YOjJ9q{!gmI!uOmS;`U5zMJP zm#suHABV|KeFS^k^ewJuE%$Ctz%jG=?y;2{&J>=e;GIod1vBrdzBU?T@BI{V-Ewcs zj<{&fl#-|GqO9Jz8@wD_exf37iIxEYx2{C5p(h^|En@pXRZ#Xt@(iXw(Su1%fVw(t zge`pQLu30f=Yfx^duUS@4o|>k)p7yN#h{5lc%*X{GZ2<_;ps&s72jMaEAZlP7T^VUK0?dCi;)sZ*U-H1CFJ57cfeLvgu z9eYO0d0psQTERRf4BfoV$jS)kcs_@ti7QY|QJX!z`lQ*%v&%g@{0*^id@I4|Y-AC$ za!G`{D%U~%(nq~$z&hgRU%PjgU)4@TWdXWFX{2V(EK9o{N=Bc#9kSN% zQk_P%eL@fFIhcuf_GauKpP*g1UHJ|9*_2kx#qdWuhTDGqC#^bLZH%{y-;D_5JXyFJ zvU=5)__UokwQs-Mm9wRY3}>7T>}*IBlAoR5OWn`9qOeB{vIU1TmcVvwQD|tg+9)KQ zd)Vf%R&)V4GCv;bISUTLG{x}F-Ny;ilB$TG}=ppT4=$IKREHZHk^!c3p+|CPG z-+aOVGB)F3%h0zeOAAE}oI8AOzL^TxPRMmMs$RK%IMLvg+s`G&{uAR4?R{4-A&fRg)uSuIgaE9|AqBD}-^R7&MhSB`WN_q5xr(ig z(Pe;$Mn3OhsEYi@Uz}xzZce>l5f9ApyAn1KWZ8exdRu%^1a~#>N$vi-$nRH8e95@O z6lJpYeW9LH!%R z%Gc>nWsz1zoz70-?DS{H-wgG0t0S3?e_L70b8U&*d_H!)if>~u&ZN~3DVmdB1D$Zi zGVCGd?cis+lUFYLy^TZ0-`%9Dd3O++B^^tS_ZWUKDxH9qHxDE~X>kWMRgjck@#4cg z!dfy}$=_2QQF(>vF>EzRopM{At@3cj;V8PFRn_1J8AykNt=VOl+Sz$YA<=rE62Wo%^t)KR3xs%aU*acK-|;rsB&bE}5bgW0%| z*b|bj#-Dbmt5l&xPw724)@3}0I z*?gQFWk7E3btV~BQ&rBwFP`7dl=~sqlNbptcbE3nCtj{5Qxu=7{Sz-VMoDpYuiP@v z79Z^KZ0;BvHX!!d>cdAUNb!5m%x@1hCqU1zzo&!6sd;NKCaw-LIh7W!`2AH)Xw7n7 zN;_@_mjVLY5u4HGcfG-KN)G{9D;TQcR-040GSyrD3#!dM($DxHsdBu{wrR2lwHN6U zZl=(VXDyw%td{d!4HrXJt##iSH<>ee2@`wfKgGbT9Ld_o5yET#%FwFC= z3YFe_@^YQ5w82JLFj|fhZA4hsf=}`|#h30R|jFuk=mFUg`D$~1%mU?3Xn~AnBlHFf(%4Q?vDgFE^GsWrYzr8M5}Tpq2&QD zW6M)#zpwE7V$baFAD|be7xNd*$t%Jzx*5RPGkS4l#Us;*n_?E+Gi4+l*^l%WApeNG zqdKTT$;dA4?1_{C`uDg@&df(=oypce1QB*_D{jAyiW$N^<0*|YA>U5jQa;=$FzHdA z-PlY>yCV}*m(J>NuzijE8c10yNOTk5dRK?ZaNHZKPdlXx0D7AI?nM$f6AFR~wuSO@ z4i$sN4i_@Ucj4?D2{EM5FhF!?$0`zvJFExdm~g>euL67#j%N!eT;Rm{gQCE?I7`D- zS@5-yE8e0*CH7m_`iy7#nau@QOKXCE<>Plp8}rHIl;Hq|lV0XLk$SKk7-Ov;Mf{4M z&8;78AMPo!G^)xnk#r$7H*!p(Dyc(y{)W7J+<6qf;6v}!*=5{B>XNsIk2r3JoTttY zL{LUUTsIdaBOUj+WD+Ae;RYkhO4D;l=;DHa-+1Doo2e>gB4u7YI`33R2_jKxEq}3B zt@HcUoLTH}>I)S{eG63csLR16Kp^y?&3?$OE3FL08e~-XBtTjT?_{m`Y@Ci5bFlf$SsI?1;D`sO))x77?6?v+G{gv zRIE`opp(|XIb<{S_pZ;?Q)0Ee$b-oe< zu~SX})YkPS*91QAgA@DG>HRiVHf8NS+vyPu@49nS4t&F}?al4g*o5=dk<69@{|fB2 zJDjtNx)3-UD<96mChFr;UY8!Y>g!(cw7hw@JC5db*+i7H&e|=rI%A-05FK9DoVGN? zLY_sxUq3*b^QfX(M7JOi9)(&j9gVf#sd29b?UaWUwrS`SOjHH3+uQ%daKZ1NqAx;3Vo{;GfZtgDAqTmW$fqU(2u3XNlcN(X z>Y61E6<7&})m!>ownmH}jYSi+=|uI5y3jxYnqKmPGiwiLMj zmu0rZ8tzhyANi>?Dqvb)_=PLVh?g6U=1c$Q7fyw~23=Lew8r|WWGP_MZV@+yUaNre z|1))0W>(~v{QuA&I@SL^azBU|d2Ekxf5~dricuUz^7&z9!_d=JY_h!&_m5!)95(eW zHLC_;HMzi^=3`sv;z2|^f)kvzJpC1cWY0)5D_ZGi`RCYwSl)l_^s$Ayxx-$f#KIPt zD?+CH*aG~2@NSCA|Bb!(j%qUd{)L^fAS$SXovJhu5fuapO+h*Y1qJCUgdz|S0tlgq zD2}3Z=>d@vkrE9dKp>$gB7_<`p(+qU2tis%Lf}5ke1GHo9A~Y&?z;DV*L&BSKX9!G z&vVY%XYc*lpS{mMv)Z?im@LJyrJA*_u#~W0wseSs$})E#KGg(y*#C0%5#wmwCZ(DS z=6&~FFDB;a)%USUp(KY_dt1FW|BBQv_Dl-45>^l@n$+X4h8j(^oJUtLQ<=A|H)0M? z9TeI|aQWzj_WX_7118`upNO*`-gfhENN(o-(Y;UEyJ1{(^8xr4whr<}DPri!Q&0H| zRFxCya)x<5ezijU;)3zkyd2o< zN*X^0<}nupw$@Nu?Vw-qV=kiyEI-$^N*>YAS+c*S=JU)-08U9~7R%x_B=>&cSmQ1a!5Q!i zSE9Q-(g<_8En-Rh3Ep{YE?e3D<-NQ5Hp(7ap+8%V4O-*w@xzVW)#YmXr83&ZfwrYk z9&3JUN)g{D;aoM*LpjPezX;pN6F>RqyEiXJ1;eurvRW=rl~9}p9{6|Oir^FLPgIV5 z@+E`f#SW)igSEMp?ZCuX!|Y@Xkj50Ne+K(!K@_dH=MginaeI`m&DP z2z<+$yoW+2pc~&Ke4F;~;(m$VBV#0Od5k{o6R|4|cQmygH5*l0^QpXmOzmHZhsX4H zKOJtv@8wqrP?s(wgQ4Xuij%^r*^OPCimY%RJknFc<5Hmqf(f}D@z`Honn)lVW@1$h z&CF{a_J)P+d}!GegiG$r79{wW?dft80opJAXc1UztElqHPc0W%s|$ML$@4K9nMnhK z4(m6aHCk8K!h&V9TEWUO%x_E5K8p@;+A9)wEo(h%V_#_C z5HU`KuzX<%+!N~8dv{;sTvz`2jfVsNKD3=XI2$^uQ?n?#LqjdlV(?PDs@7#B9OKvE zKbQQtSO6+g3ep>Pn6KIxZNH_TTH*P-%^7Th4eH+hAO@8(`}=D(<{fk^YZ@H39*P6H zbOHK!I>=Ko3)Dm1II!l~>BBq{>rd{X^%7v624jlIK$K>&C ztLn?%6BaWZcN6PLUvi8&uc^EG-sJ}>Dr_KYj!hLF*?k@RcqWmEW{kU?twL0 zPFM>ax2_18OpvVPR-q=Rv?RHUHp+Rg>+%bZ7R-Dju#KJb6<1*<&>>Ef$Mp=6cJTZ1 zWZ_h7sUph=Mpk(_dwMpBp4<1e-%o;F#t8LQE%fVt!C{=Z{44od!A8i-elY7bB7E>D zRPUs#aAz19NAlcfMAt#)@;!GWEWo0r7z*&Pmwq#~_Lbs?Z0GQasg0#T-aybWWBHcG$H$#!n9n_Gl*`BIfu zA?+XdwlVs;V}nNhaQx7!lz^pI8lu8Ww{F!;`{v+3sZ8XYx}?o${Xst z9fyNG-W9v@6sv00^#;f(bIW(+;KuEF=xV*yZlq4(muFO~hNC^{`elLLp!iR{GB#ne z@@zm?(Zo{;uj?WwKL5Fi-cOY;P^siM>AsekE->%^4%Dd^Ta%PUGE-c<(eRlmB zw$A`$-6q9neOIuMRT{Q;K}b7G8K#Q2cVng+wQHAm!f}1K7H(hac@1`i?PyTeZ>Irz5dEf$egoPpuo%BajyEpf{`$mb_E2>TbVxz|KyFr# zld5a!!$Vj{{O6NGM1PW`s-ddikwdptUk!^ztE22yaqL!guX1CLqwX`^bx4E**47h= z%!ih=U;dhukY6nvK1Uhize6r~BV*clH=aCBHEIgV(nFUHP1&IWkEuQ+Mn}#pS2v#t zB$T|a+d*a()$O0$!6h}fh3YaiiN%gV#W>GCj}F-ZDJ0%r$?ftEIU_l74k3Lh=#XcP zM_^T7?WBEU=sQ1fj0+j@+Rwx-whKGDQi`iNzt&1}cUpXDeC%DP*zGi@`QDio9;vxy zVme6p0~8t9YOd)qa=|Tj{)S%1A636i+bX`pvs@lcRI~K4&n(5luCJ;EGuH#dIK`}h zAs{Ig{z`D6m{(sNpO=%BKj|EN^|}+y`ABTq`U^&Dn@2~}4i7ZQeF+aHufk?lu?P=B z^w;X4Ezhe&%eVF;W+;$SL?ZM-OEaf1uxc-v<))fbCd6<3;PPT&jYT`h$j9ceXR~<# zMr085g(ieuerH}OU5O~N9#wP;UCKmry1!1T$S1cJ&Y4yU*SUr7;Sg;4JVVf&z*NOi z9pdp9bBzInBshaIzI)24%p@q-Aw5S?6FYRl&wNFxBCaSV=yZtP=%{R(!*H0q=afSV zT#*(^0u?jJKE1gR>nq3yF*c^pc3|Q?2Hbs!pL)xx1N&aR7>?-`C=abzRw@+kTO?i6 zP@?KfsQ&tfd;2cXhA(fGmN>I8yr<+rA@o;+_W4y!pD9r-r6Q;2wAQ6l3G>V&e1 z<`m=bYq4%dPMFl1J=Ar~wQd6{tu)ig^^?@ z?t*&MTqFHKYeGQznZdSBwadc8C}`nwN4aPDa>vwVRm}v<(Ch^n(lzQjd!)6Ho>AU6 zG}+6WOh4NbOxOc}uzlr|S>-}N+{t{>knUOd;VUD%?wp{={D3sSXf-pNn zvkji5#0LhTN*NV+5`1vZAz>s-Cnkpi)yz`fLMpHWDic*6+N^*MRPrPY`IxSAWzG7A zxUT7O9yTxW?WGIuAX#M3}>8bU7k9a}c8UJ^XdrH%}Xb@(j9yU9`zodp(vp|9w82**& zB?YYc$$_J9hQI_2I8S02$?ds6!jG<=z3u@;CIjZzh=`)smYauL%~kqWty>Hp6Hi0u z$H|g-k&}gWr{Kei18k&>+@&{JrGD3F3G3wErlPOp=&L4fh%xAswB5L!5~m58 zP`-6!2YPAbULL+=Rz-^TNSz?>jZVgRDBc>kcT5KjKbiCE4f^7!Ivj!^C($KV%66Av z+_U^bJlD}H#gtAHJwGJQb(GkpL?tN3<}wpU$ONv{T#(BOft1_#PG5BzOV#PGIT#ow zHfEny1SgIN>TGRJve&bTNnj>JP`=38B=yFkAos<>#%@RH%E`jUo{&m9G94xwBr1+sr>N#XCgVgzLpbQN0*qCV8aS6ct5}YR3sjBv zaIv#^;+|Q1`B=GQbKhm8hu$@y599g?+tJdx45w2hs|-o6EiF@hhu_c+eDofbx6XvR zOBLQzqS8OqR z7T+nO@g@S^|=ZTxC#Xt24XPQ;8h#> zg&iBSgy3a06QpB5yir}##-a&pK2U?iz6fBr$jNe;O(`o35vaUAzu7Ig~O=AFlz_sXKn{T(EM}HiU-N!O;yW-Q*e?d$r20lm zy+oud^U2fzfnu+nY-6D`nzoeI&z%mV)8@Dt6PY&X({|YC)j?5ZO)S0?LZ{`fbUGg9 zRdXAd_Gcdz<%FU$@7jBg4CPL~3G@R7-L-Iy@E{?N;z1TL33}l-as3eay@SNfTLWtO zJ$o&C{frEEe}R4JJ#J&K0aZ6BbYH=_*@NDb?*>+JO2x1FiDEVKJKL#TihY2I+p=kS z83(M&Q?3>~3ny^Sn;x|yaL(*y`f6Zb_z-K8$`}lEF=s9$o1Ri^8n|vDq{MD{LjiA= zx5bqyhrXJE39~fKe!6v1eK1Xe&%CGk^>f7$y zK%p)v!(mAiWmln394(Ltf(t5cO5EK>mO%&^@-?D$oA`UU2!HUGs^S>y;9n!wTb_i~}_%@5${f&+$L$>dkJ=Zl;BInEZ`hc6SZBXmtj!chv zg<}4SJ*Pw!O3=Uzlp0-}kYew}m%9(T_OPE+u=4k!x#3|Foo~NxT2*JoF-t(}QCvO-`S@N)Z5W*>m{C+};ezgt6BgaFwLIxW7 zCrU@itYmniDLKOCY(_Q0)n79x8c4`ww%N8K>6qCALNJsIvaMDTn`3K^RvTt6*vd{O z;++g*naR5W5C(5*)T`_I$5 zjzbQyksKeQ34>A1D&^?+n&QJ|HAt|xj!z3CXwRnBt| z{3gKd0g7gJ{+Bb~`Q@hZ(Jr zK((@*5sUqeED-^QHm1<$4Qa7hgkDppWJ%3g;l;qvrzMS8#KW|uQa42HTYc$Ni0bwW z)If+|aODuc5ggLzV6(q^Q&Ij>2HGKh z7*+)$V&$gJB0UMo04<*{wvV^b#W|1z-xPfILl5zrnOGs_p-<3LACjy(zv4IsvhL*Y zp;ncF#ZGo%&UMTgcM=ZaQiJ7pM>Aj2m%7_}6;~>7FW}K`egP<|%FMTA5Y-RMG`HaK zLTgO$u2oD<`k%B}S?}OFzi0JaU(x`RXwFOsDw{}1_6x;2+9gaOU8`;##8ZZGUpYl` z?e@)|J(YtSy@@DSZW;60NSM_uKIMjJLNGIb<~mPl#WL8uybhcU7o~DtB4pEom(Mj7 zZ#|RVS5%wE8NfIWV8sfBTfX#)T3W1o;Js-H&}3!NrZIE*<(gT5P6nbQnba+RD`1A* z(LH5X{P~4xq1kf2BnwZqVY9Ldedr5%KJ+ezb@ama~Rro$5;#JksXZ}Zk)lGZm|IT^^b_sFmZl3jlh2QlsyMsR1?rpu~MlT?rIfQRaVCJKLsaNy78J|9C zPwdX~BTn_?SJT5odqY3TiNfin>tIO+$B8vFdb5i0I*cq=-ZB*8G}(@ofnGXgT9uaD z*cEcHSeXa70dlkET~c7|Zv_)z1yzg?AG-4(}e~v zKfyjoSngsfPbN@fay;e3PIr}NHI=KcC=agThF)Dpiaqh4>*vqKO!$8}kcS9siatko z?QRn{L4cky9}3<-MLOEUR~`W-9xoICMTNF;Ayt_X1gvEpkFs|`BVB1UDOAH$I@2cL zg5TR3HxHg%f2gdSyj|f3;oE&iscsqahFZ-DU ziQwObAH$E@295SRx(+iUho2;cp!W>>cLsza`h%LE}5Hm=W^A zXJ1WsD<%zB)##6U>}kpo-Z>{-zFx^5xW-6|;ftJhS9;e;d?Q*e*X`c{iO?O!MnTEh zjqf3ljbXmlMtd&4lRHy+$Wxv^Dp{U6xY#04O+tG|(EVF4!$aW0b(9@bFKx1d`+E4A zWd#q+O)@^;*7*tNG|@A@8ze)2qw>7&b%eg^c23VPo~lQN<$3Y`b9tSPbjZHrFX$M? zg*Wl1rMcw*DVaEDIhJh$<$23DI8h+(|Hx2Yc!ir!c@42~O4LyviqpZ`4?cJxq8Ss~ z=K@#9L<}w9EtIVJ)AUT{LM$+9s_!rJ+8mk#8R6}BD_?Y?Wn!&X%AgzTlEi$V3HC}< zWZt)DobfTtSMIN>o{>DVxaWs>32uR#xADp-aBqt*-XJ!9 zEoJ@Ouj9v$Jr=1y9+X~h_HGZaV_AAV3^iRec{4L#eAyaI^mZxAJl80TDLYy$zU^)f zaEbUbM1Ak_yPG#onGkhxPX3?E**S>KIkh>lnp4N8Ov!2e~o?Ch-8;$a&G_2=f^}0;_pPZjOxfzfOdN z@DJ7FxzEY_?hSQ(ecj(jJv1qeS0AQ3YH#z**&*RJ3+lgB&sWw~uYn5D@T3U8wa47o z_f4wpoKwb{oS441^-4%FJCL;DLQnS(9j=-EU6yP)b|72eg>Fvg7e4Q6rb`0Z0$b!e zp0q?`I(RM94|$heoOVn(&s(}L1t-1@Gb!`Y{rJjT0wvJ(oK5*lyjPU4)jg|UV&;%a zYM2I3PAFT%?4w@%NkOdfT{w>1h&9;OoFtf0k>k;ZqYpnYCf~qcNiRt*$tv;6yeb_f zcoux>I!x0hQ1{FLw~ub)QdwiHJ$>K;P+jmm=yldk#FqQv8czQEK&>T({!`qdr-!jF z$6Zxrhbiui=YBvo1L@hTYwl)HX=%^s_wVG zpmUr{e7wh_{YyRLbB5ZQg$G=1EZ!W2d2E5Ptkc`{b<; zg5)-ZTd5KO2jcOG1A*IKf$FK%%zGhwrJ5qWaW6{F3#JTCk=Zzu=S{I=dpoBs6Mj1Q zEzU0l7goJ%5cl81m9*2p`b&46L_RWni8wRquHO zz6R9bG`*2#KkeZCLoIl+1s+&^1#T93-tvyAN1D>!mt?61`;2pGE2^By$w(CL%xi6Y z^$Blkp4sW4l#6Rl)=`tyLcRfx_&aqAaIZ{ntKGeHyfmVNxAru;0pj4=;S`nHbzrEU zb9^AluU*Ji9r}Rw6>@x-n*did*^zDD4V>RPKl3#71nk$b?d;fr%hTI#9di3+5P709 z@&f3THW{;Bc}2v^Hr_*NfEh+&Hl%Iio~BZ4$)}&IRqgzj2{e3u($w+paui!5lK*1Z zv8m~4HR0t)g)tp6cY(8EN0+Z??g#xc|7*zSSuxnJXA3J12yJ;D`P2UdMkn&qmjHD) z3t~T$@Mb{{&rjzC9931@iFgfMj~D-4y`PVJTUtasyxzOg`e+<`_;szzyB-L+rHf)< zWAPRu)26d{ka)o@6y_=-6l1q}_BrQj^Rl@ZHZ z6~?>kf;#M=Aw=UN`x}4_xn-j6o;Yx7Q0u!Dh>X)ldKSePjGZX`K38hiVt z4PSa2EKRK&Ke^fSUuuUT!V^$qyVErH%c zz=1;a)xXT_^V{7W)OV8MtS?ZTZ!g~#RhLRX31PZ`5_@0Ff2)j~O_AXmPbbnwu13tF zftfxyEu{P_@bmkgcRZp38%*kHxZyVJWyA@UT?;cg)i50uOzq)Q2}ufZv#6)O%vWfI zXdx3DJ@4JRxj9etKSwb^q)A>>NnFt?;HZ@`qg$^CfLc+_nW?1qN8mD(_ci0_hVa{f z<=XPO{vT^(@CZCpYwPdzZRt+XP}gQ7)#MjHZk9a>yAZ!=<>HlLOXfOr8s`LT5&4OJ zl}!GYN1a#IfIqn;@{gIN%J3_2NJfxogi`a~KNjJ*I1IG@>Udt|!^A@Y zAER#!)*RNav`&6m#^*NbWkmD00~&8f1w)D*r$BxOG{OvOrrUYY=#iuqwxQyNtxj4* zDMk0+r2G*$_)-_YZ4vZVhP_~)B5Nq~#)y`qtX?y`_ z!S27=w^u(`@&BV!ID3>sTo@kR*vi6>hFK-8=qNdG^47d0%jwPeZA>D|JJXw>>5$Z3 zRkQO7fW&^Xnkf&qTCsikGS)AJE*Nnu#Ty8<1)BHS!K}%->QOt5j@0loo37%YtJ(&= zwPdFE4CRO0ps^WMQm`Pg7a@rGOQBz$Kl}9nVDEpj9xmABF4si!na`4fAZ{zF!P!_l zj7Q-KfOdYA!i3edlV0r=Iox&fjh>p45IW22Gf-=^m1RW}V11EQs47S6$v`rF6?^l^8_q< z#^AkbSflapu-?}c-rixt)Ftc76&YrB*x_I*U{CVA*=3RgPoG>@x%PtzQ@oJf)oW^s z4ET_M0xc|N5l+jip}Op24{t2wdxYF}w!mN2wt>}T%Zm&p)~rtL9(Hr!dUubsc z+_AI>W8=Fh@Nt_37tIQp^uU>ZtMSgT(aD9N&o4B8V|mEWOwKDgnWDVQZ!ki6+Yf_; ziBqkK?|_|^wB!SSIYo6@{`NNU17&XS>k~Y*%K^Ymy_(fOs<(;TH0E@nd>D7hg}kn< z9qOr_dvh8yU;dKNsg1?2cJcTe>$$%V6tZl?>ZvC#l#Mvb8_s738gqRJOw|awjalfe zUMYqH$96}=0!cs#$imv{Jc%|G+*WoXr4Spa`WF>uz<+rdGaH+a06*0DK&GC z&O_nX9@;JL4#_sU7-s!$fioKSI83od;qJ6k21Omg)_&g+k-osq!pqf+Xd>7R@>8Z; z%NTf>xd*70zLll$#2Pmd_&UU@pX;f}5|I9#y_b)LQN4f&@!D}_YYNcXa5>;?>8kj- z$ZJ~2_B6AmlzJFgplk+&1#)c6pyh+(=X7raoxd_1Ddi)g8KW8QwICCynSC;})gm?>f!0$?8YeWf2YIDaV<483{p z*nL+ce#Dfdy$csc%^P8??gRNHFW=Bh8iL=bWWJdJP1hv%>>iERTxZHRF;U!`1dg8b zVBRBe_Q;CJ`Jkt8mE+qtmq2&=I-J>WJbm~vSkj7{%P~K&iFxdqofic_JxY?)0gr#+ zrLO0Xz~96^;)91oKZ{z>^Cr)0A?4?qnD7|LHr2lM5o2K~h*b^TCUn}C&1e{L-%Yoe z+}pZ@;!b-~NbsuPCU_BdP_4uH+~mHkXGH##!$|x4CQu~Em z)=WY0Ap72E$g9v*EdFMe;K*8dFrCLMZIo5)vA$%3r>54miMA4*NswGehQz3E2+M4|7B}^bU9Q^7>PcTj#|DfY^4498JM!UV zcGtK3?j>e)$#~jK#!S% z6b3}RS>Jz^yNK@>gdEuGpGn|;T@MtPI+}Ry?M$Jm)3|1a#!6Z8=<_o=%*;wfyK41~ zfcBmVx=#XX;Q zBD74yR2YwJ9b$AH)kDN4y)xIMLuRYOs?=88IWRivw_YY@KWO~*e@xG9FIG~d&UW5j zj^0VX7H&q8NE+{qlVy_Oc=8%4em2-GzaR&Q5tamt-!X;%;q*#i#U9*QNMN>L zZMB&E`eb!c=#c0|tE?K!mwy=lQSP9!ebShE2$)$Wxy^866}vVu>2*asI&&j3nCVg# zzcy>(KP>@hd({SohHtD&{=~7n$XcvP9+R(;Y^Pxk6T~%b*ClM*2JFdQ^WU<#W0Ha2 zwsVBWR|2U#W}&`)m0kzD37h8Wc?WehA~JDmmznjDiZC8Vikn)Vj)Ft7WdxbZxs|WN z%`cADV}()&igM;zMr@rhjo6pVSWKU#M9SW<%tHNAN9SW(e{}4bkHG>5_97Pewr%k? z_`?WlOZ)kV(Hky38+RO*|JL_w0ZQ}O&WxfzudvR*NAx+-WmO2Pxt|PG1o|1;SQEn&75AO$fp9*zB+hK|3KYCW9Q%&;u@Fz~ zl@?odME>=qO3GHZB2RsIh|a@6?iYu6c5lbyyw^VuI(#m=8oUbcs>fEuwRc+B8ICV~ zaaVOVOxui&<>!>G&j%Ng!Z8oMNA9QiVdR$0E^-IsXERdV#Q)`ak>n~n{u&q$33OH< z$oP_?Wiq`*?2UM%-0^e}d=92rD%7 zROWxS<6aU?qV|&6KVO*0i_hmM*KZnEdL{589+sb<>vhWd;MQRI7QN7x zDYRmj#JFm$T;E7J7GJF>0Gh-pcZ*ed00I=-{bRhhFY7|yZ=vI0;XEfTWXm<HUme#myAz3#J?9+G=chloBiZ((=dVxXxlNtH2r!vN@1P;o(c%&L8%7g(2 z!)qa5e;|sEI-6QhKFAOEF`C0uzswBDJaPCe>%)ASdN#gppBsZun47o|3{VvxShCD` z`zG%nJ%w*lZ$=1A_U3oXWGdN#{*OJl|7bVaD@2EmckR~Rz+ zN1lVm7%4G&l{~vU3x)1I=Ziq!wz&OY9^f^La*8CBp|1QREc;Vt5-9ONj75UaIrgvi zCzJAULrGhuz0C}=yT&6Vd8hoYC$NSqUnf4qVKhLBzI#CI3YjuA3jthd=chC1IIKLA zXa{sI{t3HB=JM`c`_X5cs2W%3zzmCRJpkq=dez*ajyUUUm(YA{)xHec_XH*2~`w% zfI8H9>Fw#(T%Icb-0bA$svadxru!&Ke#y@*{nE@>W^|}wAOTz|sQ;3Yd*-yjzrdRo zZ1=JevmVP*aV{L;YGs1h)1~>>>l@VB z))Wc;AYv-W3;glQw?7>B>YREeJEcQr@}7;*t4>$NV?YMe&N$t|fEEwrcDBTSr`Y%0 z23Q3_xU<%C-j{b7BI&&6v_)dHknu&KB(BUZ#FV9Epqmuzt*0TWv~v5eo6i9rH^?e? zdx<$?08rjyyx)nhC}X1eH8XG zH2IL32-hm}dK{y|^d5xgB(DGAPbh{}n%!psT)AaS!iReYw`@td@txHFppz4G7p>xL zQni`)aGb;t#|xetO(AAKlgj4br?*WmoIleT@J{4W%uC4f`_!{1fL!912TkJ@Kws3B zQ{VFtLLz4j3a-Zfcb_sFr^n;vcJn}6`G}@N<50UBH9Pec*dJ1GJyy%jz@MPl)eWRr z{MaunzWiPYWR8FlNw2W(VMXs<6q|njEPNJU>8{1_qN4IVKrzGZzj&3LXpK&F z+k;bk1|(s&B-H)5D1pLFIb~^yQP;f9zx$V`*$E((*p)kCI(%=Jn5gY$4q=M>E?Vh{ zD(6|8(Oqmim}hGj=RX*d<1?dixM4rAN>9grT&2zAA22%=Axj{gXr=Q#`vc^Fe>!>K z%gxWk_h0?9Ng~ky@vpVtY#zG#3H+XO`|}H+E9GC?W)am<~0hGAx<9wn)hM`PK+muGC8H(}KF zSAS*NROER`yD+X-<8R%*Va+6?-&-UbmZw|=5P4i``|2%&X5elX0I650dTBpoB?bxz zHvu^&n>TN|VS$1q+c#GpNBwE3gQA@%%RWo@^P~!Cbni+zE4w#_o*_3?Al;D1HQ#Pm zgjz497V6Xe3#+1w$$kYR9+M|W0BNQ&Z|4LI{U$75=9G6g8nPo=b0zNvUoQ5DaD-pX zy}JXAr=EhN^L)b^gc7LeQZ1y5&+0qL+M9KLhq^UE%8{_SRSE*13Ho?N`D;XovJ<&09Lrnhmn|5u+_h7-u@8E4f5V~c{;iuns)ff8dyT?z&T13;DJ>@54tc3uo*P8%jOa8cY@19g(&+Dtq8XH`4!S)8vWTGAs=ge@lJyJZ zB{`wd}h5F>j6^$xqMrOuY$5olOJlX@`#BV%&GOY)5F%^z&!55Mtl|6Io z2l@yH)%D1cS8SH&DX&k5xQ4cQPeBkKM(P3p@rI*gv6c9be|WY17BffA>GOMLET|#9iBEM%!>kT?#5}q^DI#nZ#qW8%mJ} zC5?fDb;-e8+?ylU67!z0yRtFR1G)2t<}3UGd4Z)vRSuWOt;Jq&f~->Zh|9^^4%%E^ z^|eZi*$W9MmV&@*A}U{Ri=xj#ZUam4$%hn|Z~qWlB9%UO6m+kV2k3wknfSpmb728G z+N8!ZJ_T!TACdL#=OQ-^TP@_f7FS+u7XeQ8{?ZO4ResYwI`BH&GtZLIoFl$?D`{?W zffU8ZZP;nPGZm$uci_xdUF|dm1~A)qph3ksDWKs|BSjK=wq3HoR{k+@Pwso{aXLrN z>YOrg_V5reu2e989{yFMK;C?XIQ2o(HXR_I&)XZBVVf{>M;rNpnUj!cxSM=__G;`O zTV%&~rHh(3UKuS2kTk*T|D>a4*uKleEogI z(#v0$dKbFKL}nHbGW4;nN2^zS{GRIK+DX7mshQx$VYu)C5~EJ>i#j6cacnN7RC%4y zqqNym9W5MEeJVWj^P-{lplQp;YLIg`>qXLY|7PO$M_|1e<5-KNPaKwmFi?=SrDD%_ zbU^tWEp+Ne-tztGs7lv+xf%NxU<32c_Zp3-`v`g4pGWrVRcRs*YPIO+W8g|JaHj(` zX+FXE4}7_Ezdr_gS)YvxVtxt^k?uSBN+PxKazUF4474k&=D>QJtPwgv$y;{&KVCR_ z)Bli>*~y+f^*i5QnPzr09a5gMgtQw|RsKWKD#MjqHG+-g$??Iw>Z+uIZ{ewDZ($%! zw~ZdovwoA$R+gv*Y}7Ps78;OfHrY0k!h}3qJq)4~?bHpZL2SvC(tVrPbr3K8<$Qg> z==!8zR7u8aj(zs_sMCcpt+@wyhQ-{x13NT34CQ_FJH%;wT4`)oIT-utjv<^JeenF_=XZ4OmfH*q+LuH7@~nFwK92U8 zNnxF66jjX4&F=it=2rL7zS)%G;M)9&%jb6Hiq7xh-ZGdPk?neq4NWZi0N>afdrM=@ z%Ih=U55$y6lFTZ54VQ08U4|oNUI-q`r!Gph0J{@tgQ?PY%FGm$KZCt7pp--O+?UgM zyER~N*fHj%wa&93we#L5S_l{%(`;8RWX?dv)hxSf4E?&#Ia&_VEwokoV@u>zLL*(9w&=2%X4}V)_UnefYVHA<^#P-F>2Yk)S7{7XS>k zB8LLf=@#?3Ns>>hNYz$ztq!LxA(QLtsb^w<)pdT%=YzJ^NmEi}`#1~|Z^+OfVXa>q z%beDb02I%4X#Cp8)KyfBg6Jt>4c5qbBt~()oX3 zaDVdr|5KR$e|5b|;<;td4(vMU7V0dOQP<|KBV%h5j}TD!QztU2@XCf}z*W8Xm`KNg zqJz1?$S!VyOQA^UhP-|Vacwr$pjSHh%3tR5VWHC<9zOsYrSIN**Nc(*=k?ZOYe14W zQt&*znN;>rQRA$pHRTvk2rxI=BT>;C>!|>_{s>$FlxJ>MzyeP&sT^=gP_Upbt=X4QtuIw@sMRdooo=WqO2Z&nymXZ~ikO;NMqs)-rLSHGdY|Aa!4g z!~D><{w$C_SOUamqc1-IjoZ7}p`r`}_xGnJNmSe+qj1GJ#uJ<44L1hM0WAHxgsEjH z6P(t!3{Vd(@MB&t%R&?qzD~zs4m}*f&1hs*impz3mxl>sMp^QnNAEY5)Z?bG;xD|_ z;m5lYs>VaFZ34>n-pWWYhg39;EfE0aeaAUYFgP4^v84w(b=UsgQt#)Xah&aeRR~ zw71GOSVHL8i>1{Mwzyol%nx8zP_C|G8Z0*EM#&Us+)|rMH%*|m1o+$KE zHf!KBQ)p_BIo!Xf?eHV=9Zl9+X*fN@-vYQ9thN#qYq3ux>f?ulHq#k{9NS3RjSG#C zKcbmvS%KY&JaWACjbDCgy>@@rmgI=*^KUtzNv|nAQQY{=u}$1w6vrK3uC^VMz;zaN^yZg=s593VtJ_1|bva1@s@quw?Z)^Ke9CM6Mv(H4?x05@D zgJ#K8+-#fa{aQb4iw?B*+LmZe?qKx3sO5^-C-*ka@zm+Rb(jwD zM~eLRSO*!yw4PN^p5;^vf*SNXcSL!r=(E)_g_`F!<8wAicSyv~$?;sQS!XJ0Fg~jK z@|M3m+2=iI9%n{Mgsu9edlARr{uw(|rcO6SvL0%9AG(2*?3p`isk6zFe8_OSiR#Vm z*&P}UVPI8UOWOl$BnvMnc#gGP69jp}ybc}1YuI5``U-t5e+>!omE=uY_IStVHdAM@ z>N|-ev;l>)8}=9>_@}q(0_VRi>rH}h2TXOh&7c`!zt72{WYL=dm?>H9kM_3)% zc(`w-`Ud^b*u3W4gicmhcM=4pwNE(0$nzV^wfm)w%a;8@Sv#JK3?P zX1A<0rDKKG6{^_I3?xezKO&xNO=V?VyPMa5=rNBD%f&r-b@BZ&jlUI3)S@BMY?7)6O=@H(M#Q#Z~9vPQILg=%dx1B(8W6BVcHgaoaq4TXKu8C3G(W*2b<@7Kk} zc?Z~f&LY)$5u+!=s#so<~fRcpW9n?o`d%Ij7)msPvznLf(MwDY@hCIZ;V`Svi6c>hS4PdNNo z-V?BPZ)c{7rkwa4q0XJj>%^CKtxV2IoS{}xZSalW3%HgX;j*|j9I76to7IW&yZBNV zocc~|vMTlPm(gMcV=gY*L+_Yxo8(gF&`ZOL^*Q>MAmh=DLnCy2wxw3nz4}_0Yj3(V zf%fXKW=_b+daJ?P)IFIkw;sOv*bY$2#LOeNwUINbhmW4)`;K{2FHhC}57yp19_sat zA6A_sis#e3&blc2ndkTBth|4xG zZng6o=tHjB%3wWEi2KZ{ls9}np;@11zpeodm!&W!G zu=O|zV_zm&2dK{P?C;&XB~qid^gZfmM97|hKBe#exL|47 z>mE|t>z5gGs@XV==wf|1yZb{4{a(EJ+Of925;CBveq&FiMX!(KpqCnotEnC$eG9`3KN6fV6@i^|G4TmL5OTw}k;j(ugA@Z6Bi>(5q?dYX)`oIOPJc4>W4jVKMk zqo9y}>skAap2`8wtBYO^>YA_3$3>Rgo+jMf)M0tyP@56{W`E4nZ{^k*rZ}>q#SQlD z5w4#4b<3^*WvQ=w%eB<`A_RaEQBNRvky$Pn<9Gi1_;2_A>zA9#(sZK&BS zNAWU%n!2~HjaQ<%!42(`dr$F?&Ni$}*Rf0aXKEU{H<5GGXbo?YgfB{4dz3&1J(HH& zPy0ic0YOcksx!GIDgdOq)O2#7(AShXefkO@r~ZB}VjOfU3TcYXWww{GBc`@2dp=Sg zV{WOIn`63)F7du6rZ9cyqOL`cpaO0WEEYR4U+mb_V25Moyk)5cG0v<7&M>7NKicm5 zm~GPtA3#+Fq7*OHNJrOy39XkTlowO-Yb0v@Sy1K2t~Y^Armj;F+GDtcZ;Rj`3d67Q zeLkQ*!2u)9kO!9odh1`_TCRgixbz&RwSEFRgK#4|WN-}Pd*08d<*Rc-TOAm$<*^55 zhU@0+JDe2hC&tr_;Zf4?=cn3@uE)(u*gwwd?P@(UeQ|3PEl0Y&KC`sH`Q-P*4qB#| zomHVQpbGOaJOO?T{^>nF-heMH^xar=W3CDR@H^k|s2uZ@1(^vSG4mxhZ9j#i&MrF9 zK{-ThDF}uiZ7pWW1xHfr1E{Sr7R_t6H%I34i)P^3T#pH7CUvvPF$l|ep4 zqu|?f7wH0_hJF}qlpr7=TRepePN--|IV*w=HdpCT7ZnrbXUroDlcW6ymfyDiVhz1I zXOUr=@2SXZvUrNWg5a<94JhN>$SK@lI(uaH-dnp+cSKIjKs;al$7Udze|s9L)_`T) z_WFKZ#@ycCM)65^>{m;v8awg_sm-O>MoNfbeg6S11iuu-9~5-ufR;H>Oq$eis4mv+ z*P@Ahe)YXDP=gwnc^I3~_~dIXG3JKd+uplo{q(e4<#zAStV&;w%JZUcFahuwPB<~H2yeu5ie zcv~j+df}GcaF@mwh)qHS3j`f{|IHRXN6!j1_9zGOX>B-<@N4Q7TRd$WrMrqJPp`Xl z7Zne43dPSs-QZZ^-;VZETOFW4V{a{Aw!(bz!!h*{S?lA5-qj!$Bw6S%mO&`J2GZN= ztv+Xc^~=}XS%wG`9LHuIzeWw{JUc_F-R`)>KFs(HYxT=ybxm8iOS0O7D+hustoSip zt@N^?(c@+o)fke7l8kzEdktg$J9&!R39GO~`%zWiQ!1o8_S2uz5;lL=+CaKx{ zRtoPP#f)qmJCzo6X8^94cj3lVHo_ct`|flFVCslkG)!IjXibAou?->$f-p?`JJnJF z+ewDA6HBYS-_1QpT$ywo?G~(Pu`aWwdU`ktAWV(QuZfZ)-QH*2XoQsn+|KoDwO$2@ zOHK`Sv#TTx1v;J9t$dsX;{UG;Ba!m#_LZr+o!yeDQaA1KHWGYZWB~7 zndf+5-P$J!`olx(@SyQ719-KVf)els(R8d*$a$^sF3Q+M=FEDw)Y5A&%O|plHI&)_ z_a~_}ze;K+2#mXCuIfX~%>YaA7GqI2j@iiri8+g{0wABB-!EyK$|`x`l==^3D)k!z z4c{EccZ8HO>7iMY;<2YC5|BLvUDCZr({N0OJ$cf&xV+>&>bCC2r*p*fjZ*c+L+G?U z>LIxSHkCQ#dmViqH! zULfI@oz+JbnryyT&UwTHMA5g!;Ag&BvvZf&#x*!V6#X7pzozEw_X>YE-vB4yC-YtjO;8%(LjIJej9zLkW`|J@D#$M3zb2CEa_2{rk;-B_scL ztCVhd;jh?@eM>X9ZHkZeEqP2iy~=?_efzas+&Wm5LqXN&8e5w}%GZxIw)}8VSvmFO z;h%fx$B6pn_aCt6e;e*ypM0~wEu+{z5v3LSi+aqhER7`Rj2@-bB(Q0A!VYg z|CAv?wCPtq!u=Vyj%)5Q-t{#%``R3%IrFpG^P{*{WFeTf0evynLVNqEp}%oO>_+RWkDF1m!BEl2 z09<0ieTx^K4t^8K%G&D60>{b#CiA9;^u^_1xGaa-L;CVW8lz;&!-F`(=yBy8#*dIv`dFecVF-^~@)zUvQ&C zDji(T>o!Cs05?)7mRx!#q11CEVAq+#4IlhgF*MCY&bvj!mCA^yj(t22U1KA6j~THR zvJ;gRxyTcTtPO%c7x8nFu{y4I1PV0tS{y8s1rXT>MN^wMz3H^vr@0yEGr!>0QNXSR z3IrFl*H7r`#|v*#&*=Z*wp-a5mpSekTy`fHcn3v@Dt&4?MgZ2~R~&t+B=JSa;nvzO z>wC$ksG2rr#)(+q+Nfs9rLM%~?fg&jPM>9`s=jr*;AODST zD+EDzLG=##7t~q7=$;q`i0|VhrjwWJJ7{GIl9IXmzZOVhZ zD&TntJ$D8(|bn!|smvMW3xsJ~Oz4yiTG?(&5H9eqRJW>b!JT7=_dm7IzbpY!559X~J z>}vpUSJ*&e)7Xxa4Lc{9I9k)bzX3{QpSk(o7JwQN ziO9?@D3}@x{|mH)zXIikb*6ZnRg$FF#)s3{EiYYyXZu zd6dRU@769r>=sBWV3jB!JhpS2GruvPJ+??wzlnDNln}4<=J8tPU-+e7zKp;5yG=hq zOr0sAmtE{)f&OhS7yACsU-CPjSNMMbFOjJb&e9W?bGT5GRVAKVt3SGdzpVt0#WdjROh<*OzZ#76czNeky zl>dVz@kGHym?t%Ct4UD*as72_R?{#?j8puySCnVuw3b6T`#~LW{{BJ<>|{D){w%I% zcI5&E^lMiTlTYA&4wLWq{6(f1SybcJ(0}2ZA|HcnY^!UIQnOi_5wBU+c0d$-rw6aX zAm?!Fki1Pix#6NpgF%`APUg2w@4(espfC2%X-^AEa5^H=Xjs&lf~<5No}LhH|NXa3 z&DEyh>5Qw7>cPMQhu9XJuSjooo?A6{LyVySFs_~dSA20XR3n<~+JxXKK@kjQ9m;eB zFlkA)+&1Jtn8ASZfm`~hcpR|MWg;to@FW2`2mkg@oV&Y+n`DEkwRk=(0{_oHcJrT~ za5p?so0CWK30}u)aLQP8%sOQ&5rAj(04fmw@)_YKJKla{ zAG2SZy8p#N>eI544DS3p=U!a@3elc2<`DE8DHtF(HH;FXo;*Jc93c){f8H7T{9-DZ zA9An7{YhK*-jqQr|8Y63Tod_Bq^j}L z{sXuL^?GXjtmoke!57;$y9xS0aNk=KR77}2RJl0A-!vd3!vvB`7aMM!Lwn}o*yn8n z<_0bslmQU`bG!hrqOLIb$h#w{mDdq4Rmd-pY-p%;>$bF_P=jG%GDX!lAmr%z4XIqr zfi>tqLqSNh`gp?k9e(pmHcH|~ddbE6vP+^Lh}Q>2ndck^JC+39I+m&qKxH>STutPn zAoHg49`#6uGhH(Dn>GOA%i1qUwFXRch9XZA_`iPpFo4Z`lG-eyB;EnQ9npU(JZSx= z!o#+-T%_&BxoV6M3Nw+lA<;@ExtD&MaTv^gQz-@a!J>@@0t^1(M@m>9|62MJHRpsVyk7~_BWp9qn97oDmG7}-}G>nAONly z*oY7RBZ7r`xJS>9Ye{cgCrhl=a>*`rB6n<9VaQ_!7&T?KfZJ;R&8TnMK9Wfh*&L z#&79?n}-a5t6sLR`;o+)Z2E>OR4BRxa0v#PR^K>I*JO1!T;b|Wh;Yh(r*$k;@g;{{SNib$=@= z+_G7=9l+xr}>*5o}6r?cac z$)z05`fA?f)g#(OS1*o1t!o_7QOIB`PtS!YAsEMqUb}(kcZA1+#_9vJ$JLMb*+TUL z_G~)O-)_$oG*QfL(MpRx79?;?WXVhL(5@-^luSZljNFG4G<5tpguV2HHWpWPK*;S? zr+VYQq+?HwdwTu}&{;VQ`%h63|O$11Wm11ngkPINum%;HG$}h z(hlfW=s!tgf~ZJSkv(vWdp=9;|ZCINzkmS3HYFSj2laDJnFp zXoc200)1lM#itvHz!9WuGjz3a%|?hXqi0e1pZGer+TsH_ZU8X;m`bB;Hq)zSKiF0C zPhpO8yg6=uE>zBMvK&+6oA|t(ADz_;d)XLyth=!u6mV`>tNN2m_8ex0GD=(j$%WjlaRca@9htaqa|56XzyUxr!F&5;7Ek`4G#JFf@%I1H z`k3-salWZ+9P?p4Qc#n49s?%l$13EJo9@Mv*B$bhHM%0(L)*S>NrD1rNYs{R!R5>D zmF@omtKZZ6 zI`({C(^f+u$H+gX#Ie45>70$FSWc9T+CRuNj)#}+5W(FF^g=fqhJEDFbI)+d;DU>A zWr?faG5pnYmit7_u5B9uDl@055X(J`A<}G;-iVTL_DuaJ+(5Lc-iur6RaqATXD%pS zsB=BK3#q_($_x90l+}Gl*O!#H0=yx9N28UEojNZK>WlCfsy66O38fsEZ?g>A6YyM< zun>8zbn{5C=a}*{_b2&R^^OjFZv%b6+gtZBJi>=i_=kM!UzA?763tJF3>n(2s+a1q z>cbC)(SHf7*{YXat)apP{QQpb$&KoqhbRb4ppU`)6qn-m z?UCin(Df5-OJAW1#wltXEGcd+dW)))XmImecp2S;MZ!(=*z)(?*d#!go_+R7PuZ3n z^h-I9O&$2`(91L3d5L+bIh zD)r-tbxPB#RhRrY!J|0_ywBr7k0bpVJ~&op$eFZUYG#wDFW%BC_{YYAKOjYeV5FC2 zb94EuM79>ei}dx%3g_m^)?x)#u3d5vKeh%t?{eb_q#0#S9o%{_f9!I6S?BJ~(-8GTu$7ED7sbI>-u1}Drn{M(#Ob+y)C zPDat?N~Y;Gh@PIz*PvZB8SVB+V&R4tXnv4!?W3WRjY}ndRQe>}y&>41U@&u!?*UoS zQ~1ZFTMm6sAtFNI=!M8<9@XdGfiWsXRj(S*ItA>3Zw{y|d1s_&7tv(; zc|$m~=!6+tUh%~2Lh9|lyu+I25!<-ODz)$SH+$S@6`DUFu>#)F^diS+UvxSQ`B6kl zS^mQ{mq(hNpe~#2Wt5Oqyl@D(U+2Ib#YUN|q0nwvcJr8~#IvEswGt*Y^=uE{{jnkV zc;*G%8DW3q>1_8Wt8GIj#9PfR-Qu($)y{1-nnTRqoVM=iLB|M6&g67{6@9ff8~d)U zdK-=PttTjR*6|W^qHUF>((!MD4_*xpKFVQLR4QaAO>YrbSyZFE`eraRk-dKy7V}*b zD<}2|^aX!AXt=D1zVD0kx6@dmA(~9+8DA@#`0|IX#wmASXKvOP5qD3R3dr63-GE9! z>8Ch~edawe_DObm@T}T9Y$>(?hLf;aC`Agf4$YGNQ`zn7MX0i;haBzoc26@pc-Ixe ziFy?RZG@=EMt1EUQgP>@XSI*wU~$8~MC^pl)3Y?jp$R&C_Yk`&=!PP_>@@7(zN9~R zL3geVntpYsqa8NB|3fS2i*Ygt?6AM4{GclWd7}>O>TQbMC`wkx)qM~PAg?ZT^*nn!ES2XX%h@KI9T@uAtYt=ZTPEvSWizQR_=s0FCICxNJZ#Fc9yh>&>5gz%6bMnz*cJ?-O}%JFf4Fe8qoSRgYtC#%F#d1PDOgHa_YZoeeXNp z?ib37>dMkFS6zHHp1*%0JqFpF{w`QF>)k?$&e_Um(56H?iK9cA7Rhy+{p~AtEcNY> zDe$JL6`52n;t{Sf>$eyMzuVVt~c+Ux^yIFU!BAIM>huTr9ea`RUhL% zm!qsuyIvDmsat9(rU>jt9nYA{Rm@_no74VMN< z`mFVxtlF_lHQDL43R#-{Cm3^?Z0ma(*pXJ3sPLP8`Lj+q_JEe)tBVk|9sly3`rt6J zifqyN>@x?52MVsp|0v;Cd!_(HQCq}F@(P;^UKhe}xTrD@SHETtKxsS;RMJge!Ei_` zw&Is|n3Au`c!jqyR$yHxTW@n@HcK*3r#TKJ9xKp)m6xP(De11_oil|G)CzAUGyRS= zzy-!0W%G%YyJ>It5i6&jVmvVjFYfVlJWL%EY^az<$J)nZy@$UymNrBv+&L&bazA`d zZwyCnpD`c8I;@pt&OM?!_B0aDsNVIg{91yB;bu|2L1!nZPZpP=7G51vO)9Y~7qL;D z2})?DMwPB&u5Owy3^+6GF?*+Jbs5;IhT1z&+WkJZcF|fV8)p4JDOV~0J`0L$2{@qT3_l4YP3kyIdk9|XR*Nk!PaD3z!@Fj)d*z2$H*SjNyiTsvcSS0`JEMK+J*RN-^g#b!@|Gixe9wkpFoMq0 z`(vNW6pvR{&4w!YU5J!qc+Q~Yn>JoH=T2JI=>@b&4Q6I7>!re_FvS({6&V!)E{006(Tyt>R0%P++I^YA0hXZdQ7B zy_k|xArP^^lt1oP>LrG2k)F4$rM&ji8XA$Mx^`3zVJnDa=@s8~`Pg;zOPu4QA9I)X zA9HukG!cAI9_3BD*l?9ve@98Gnfvx4zXhtaPxT!^tL$6s50|e33?*Iwy>3tFKmfdH zTi!effUL|AH$Nq;E?Zfp**Y37ST#$ZIVE3tBEhWca0ti^ue1>MUdIMxX`5Z>)-EVa zgiVxElE^Rpg*ETKL$cPh_lr?aDURZ7n>H%2EP<C6>n$HqjInX@DkKj zJiDFEg0iYKgH6wIk+((0o7}*^9)F!9tu_TuA)mvG9ru%??CB1dZhDnU82VS&1KgAj zNk2eE@zD`Ej}lR@9Xv}QK$FE~D%;uplGSkL{E1afhFZnV@lu^)&CN@i&f>~*^y-kt z@SVGzN9>w5^p|++AM%d=1hW2ASwGPkcMPi;w_tt#5#e;A6W0eQAbM&)AElZSok)&|7@Mbw-@mGCdOWS@>f7uI+z^Ot0)5p3^<&7 z*f;fVLto#2l2$htvY({tEOyEn`IUNOxtVkD-eBc@Pv-8*>DB2mj&GH9n)dOSCO;7ig@(Fp~~-?&*4WStj3LwALb{3XZphM8$xRj*Mb>Zt%C=M-^Je< z`Lphw{0jEHF^>;QoSoX(X4CKfSe;c{gB*Y%Z=`n6*PHR{4g*eQZINL+nqJbAtz8D} zAz3b@a=18_<=U?{O?SQHsmn+B9UFg*;$McyN}{zf119g1MN9Tq%KhlOTgEd(bdQ++skP0;ih~ zyB(PeG?;bzGAJE8r9OE60gim3zaKp}IMlt@B9z+n1+V19lKZ8+5j*&Pg6 zI!KGwRk3sf{Z`B1m#zBf$60>8utKbcGo*5edQjl{0>cS9#00w{^H>gjr-)2tY(Vx9 z_(*~MW_dXi zMfWswqy)va8g4>F-->M!FMIee7v0P0Y>5sEP}TW*SBa8nEYQJDrSaJ+m;lI)tBEzI z(HEe^F0!+}N9*Um=-`IDY4KK9G}~T{T7;Ok3Bs_YhwZ~(&W}!=aA}Zrb-x;6mFlsh z)MdWiqY_?Ml1C@BER;6am5+{>w16sKBl2#&p}#EEO0d!GLr0q(@!C-iEjG=|*&jz; z7$ckyCaamgHX#6xt>d6X@;9_GT+8K=4bvM@J3J5nREjLS`uHxZjz;cRke;ru*i5?$ ze!}p4O8|%46j{l4wxNsXsVXlc6}m}1OD2S(OGxwZMFW=Ao-}Nc$u+Upr5krL*?rUu z36|H#SPeO2NC3{0QtEB1frLTXuV?|Q53YT%4^~AV_QkxXXKP_ubL^nteVA77K&4_;cr zM}Idrf#Y8w{;gV&P-B5o*-mY|C?aI;@-|r8QaY0)`PqPaL z+l*#>${WH|;CxNCVGQBb2RQJZVAXYipnrFOPFOeSZ?R`e!3+oxMUbQRoIwS4P6O^M^G<}&E>Io&aCfqtXz^oVwHWK-oUA} z4oZi@hf7!47FjJXeKc@D-mk36Lw@qB1a&d2w)D`x>=Wj(pIBH8HvR)?UnnO8_waCxkcC+ThZh%7o{Vas8cnBar8 zaA$EVNEqm>Q+GG$X3LJcrpH&T%FD4-7fku%>^4F3iU=!fpGwcbS3cSmkJ2^fN+$4Z zX;*PAsCK#dBbur^>PVX4iR^AtcOpt)&^~exszw)I8PB-j6MSf*Oog{LYKAQH99|EF zzpRaNdITXIPs^n{mCdEWR)eaw2~1@QyO-V8i7a=OqMFd=rYN8NgX|&ZI53nc4Rd{p z&4Gt&B)6fmgYFGDVMl{<7dcra=dJ`oQ0&GkmfN0Mb`L|`7h^K0J64hr8w8FWSn6eC zCtOUKb9F@R@(2Q?W>&>ub!f0(RN&+FzB&9IAsa^#sWUTR_pehYvs?rKYEcT~pNXXJ z;{2OG0R4hGT(l2*u!hO>@_@c?7VKX1F!Gz8gSus55Z6bZx2bEWMU_A9lPQ;R++i|U zUIpFZZMn0|@lGwM;EMWX%@YQ(H+ag58~8RXloYELLGO}W!(Ut8>Rnl4E;;fAEO2jP|Ca8!Wgn?()p-my>%%FI<#B&y!EPIeZ zEhgY;8gaQuRaWvSOFw=6G6WGMSjJGa2_^cVrqR(s%B1UA@^y+R0KgU=`-QCpgDr>k|4fgrK{;0TKxFFGL{pL@!4N~T?m&u=O|?zRfr{->HN#Byr9PIoUSd{1i|(A0rsnA zq-svC9rm_IOcnfiV0(2~&#MZ<< zbDQbq(&m#O+Zp?~SqmzP&#H)z_n5n2H3*FFztf~dSah$|gk=flQ0MITpQqAWnl`X% z{WcwK6Lc91`CzBhrc47aIXy@eNc4UWvj=3%AHeYv9?W#Uu^W$rIU)%}(hxPXty0I) ziU-BA=nKj$ZeCA!31$r$=*%n6liis1L`+2v%+;5WTaTr!5>%XpkHf(loD&u--MR

k1aioNxzIY244C?tyLrGP-VtD&DP9SJOye|x7- zbK$A+7eS};gJi)jSWjlSu=>58^Ra>n3;W`R9-COU$@SUqkNcF-tZOw@ZWL~WdX_`E zD<+n6;+5*Sf!xFD9$iou{x&2qTi-GRBw%Ij4J1aD1NRWDA=2HjM=m9ZJeju?S8^q2 z8tw+kDte3NXLCx97Q%_B2-EV8G84ktFT~$;yI|4~Y>W}* zxIHc|J7m;C{O+JrB`Z20^Ks`}!UqMt6pd4>Z~iUWZ=#}Y;~N3_U?KababqJOzHr_> zh0LfcGf6BpP_=Xr@=qwkwJOcZNY8SRcUn-E)L4Y)7cq6{$>vb`rewjs(U39CA#y** z>5+a)TDsXEomO>+Lj)7O@Pn~0ZlezFSFE#dl^d`13ZMiII&aXPn>%K3T#hhpA;DKV z0K0{hu)XKC0|}jrfb$K;qC5{(jvy*r2b6kYURRrB>^4ZE5A_CT}1mKjF zzSt9i4ds&c^!Dn|szwty3sq8Btky{jcX>7<7}Fw)&fR?2KxfT;faeqxE?;Xq_{k#d zDjrTEMt-45k3CHrb7T6dzV3ow%gcfLB(M+9X!@|4Su)HOYLM@4f;u)pG>=`qZXBrH zejE+*CeC6l2=3OS=}bD%Nlg z8&=9nQX%>dF-P7U{g>xeQ&P;OKbzkY@85kv6-Lw6t0n%g#eINb)uWW2_d=8 zLx6T0RKo2+obYS{bhiq|?_?&u7C#>W#lzZODWR}Wi$6tTxtQclZveNM)LJ`HYs zR~r@4Bv?3xq?gSyi!{HP4a4tWTZEStw-T|l<#r7tl6an;pu`?qt5l-NBC`~QOv8d5 zHucem`m$(oHR(R5kRY!o}bJ}k?!9bd6h;`bGKr-?LU)i{^2TO_5H zq+E~*jNwlg0$I$f{ii7dx(LV0Fa^3it5f-iS0;W7&)Ge{Fd%l9r|xb{8NU&eTk!Q* zsPWAB*}?Ef`%rOsDP z!kGs;wrpwQT~5>r2V>V_#o{LD#k8E>%8)i&zXXi9mY79y;=Yy0=rpt3;QWcOC(AvN6VO1X6^YZ0{GkMFs2Jny+*#;Hl%NWx@mPHy|CS3e2js z_b{hCoWEyN>rk}<0OqSm8ioQmkOsdhEelf~KPGnH4ox}7(W1n-ws=}uf;deX{dPW9 zv)R9Pj?iZKiP`Ti2~VU_k}mGrC-A`9r`CEAd|O-Crxu^Nrh|!mkO7`!O^2Ee;%qAk z9Q={VqsIG6+79=TYs8x@Hf^e<&eEa6c~Na`@VQ0yXZniP@mgffIRxtvZh(E38Atbr z$*o&dBIgR>oq!n+XUTh#t&v5!!|AkLBr}ihR_}J+^bh5%*`&m-RxLv{+kD(;wXunu zBvQ3HO36{2o6}{#aB&uU;9URwsmBJS zal8Qi4>6a4*E)EEKax-9zyB7=gdHrX zI?|O`!JcJQQsr5fY<E@JN*T?G7Vw68wDGhr3B{&Z=sDZFv%J`;k$b%GSIC;v zK3)B{Im38Z>{7J{XL9g^TW@~>D^h`_dErwVP+}ANT12JF-O$cLAdlwJZ!gnHYH16_)-EVa~LpTh`(-T)Sweq{h11gqhPI+xrb zl(^mAGs`&K*MIqxmlw$t*0izMpoDOPq*|*$CSnFL@S0T3a}@~{QI}uTYF$rff8eX8 z)})?(SoZ>W6QVZoL_huYF!8x;I(2tPChA$XuB=xYBD+69(sjNk2guqnZ#j`|pnRx! z2YUqoZNgz>KS6M5v!d31M`op6Ls+c|fw@$e5g|P45OMjC-XPdjyXnBmqf!PqpSov_ zf!1qEZ$KtVt`JU1&;Z|^*=_N3hbNy3(LH?2*cy%{Ln#+ng^CAVIn+9)crE7sXL}B^ zB26*2R*}MTdQxQvPiCiS(mUnOODDqs-=yET8X`UW)T?z)^QFX#Xn9`U+%8F@e34s- zIOXx7wAq@VyS*@WfZW|UWB-Q_gN?Jdy(li&IutGOoM;ld{~hcmJ+=B{4ysI$N+cg9 zE5$Xl3u}sd(CcME=uh@fcGsc@6)1GO3)+N*F7`#%h^CIeu%zwvdqS{-n?^5LgA(yOAXErrLFaArMBm;30e&15^I_hSs%I&et zoySjn!q#*ix4o$8G*1+auQ+jU2l0TPLk*GtfSH?aX(4l1Wa(t#)J&%n!J-}4GohA0&bm`(l+Ya(lQP(u<=O?FJ=%}u(~TX#K1LZZ3>kikrNuq=c<<``ARCfl zZF{&a$!?ZpqN*>#5UPAsx2_QTZ(9`A)e1UN!@(5Zes7P>?a$4UwO$?M#Vm^vTXaFF zn$5Rhjy5J%PkY?j*EyNtP6)Pb1}AjI-3QWHTl^`!Gj2rixwOcYD&z)x#T_cGR~5$G zD#vuTG`PD=`HD(l%7c5)PG)9DxR}yN(FVnD!OUzXQFAuP>#FqafxQaO!h+W!lFYeM zSkpjBSKiWhzKp#c03_)*DZlneDDVIfDfpf$G~p!_QZI=bU|yrUSKPN+7`@hbw;tiTjC^0P%Uwwv2y zv=#l^?^VkF=N+W$HWb4EcyoLIm=H+&p)BcJWR{H{h}MM=qP3LFP5zh5Mh$dl+RzbQ zf_)TkiSq`laBKDZ=OF!(yf;zzR0_(9iLg8)h=fCwNXH(|+5r2g>-TShY0@heH!g2ZBLZzI$>Q#qDs5sj%eMljpR7xfM~qZxbzaBZ1de>r_oEEe18agrA{ z`QPSJ51>pyv-iw@qUgB-(8-^=45~x2--;$ry`sczE0z~l8St@nTxoKhKVSEyX4X~^ z&JT-QlOMb_qWMvKlX~e^w!#?BjjYqO1&%y@@I6zh$H(l#w$(*b+P3pxavacS`R(d3 z(TYZTZOTK~Lm_uL7+u8O7nL#()T48>wbKstJO@i6sm+;xsLL6cX1!Z|mSOsz=ttiY z@WDyZ!mY%KzRLEeyN#TtvYTN9<}GEHTP=(f8`Q31gix9DqrR}K3Wpx1^>52wb{r|y z6Z{rfeJ-xPFn`OHs@26Z7>V^Z=VmCrdWz)S~#G_ER-mz<_znlK}M>Z{&^WyD2?C{Wwzkx(uc#DrF8w zcGgQL3Ly43P3ZHwP^XSUz~;cZk0 zgvQ3R;R~0TTiJL7_N(9KO)lHMjlu7o@MEz19GJd5S+&o&D9g)S$;JhFuhNEd=eh1F zEx{{yES5JjsRmzbx8iQG2366ZUf+b@sY`9C^&<;4hQ%Zf&}2>Jj6O^ICYVt?48=!5BMi?Q+cu@Ra-CG2&# z5KS8D-+D%v3y~!I0gi|#>VcdQ>VIB3kL$Fu(7U$fuuw)1G~nF&Xh&JdDHCnZMHPhW z7XiQ(WKU|#N&lIg1>;S5J-dzbMiJvpUVtgkL0$edl)bk*SEvz(Ch>A0ahFc=KD<=y z@_D*!*05}kaj|k5Q~^*-0jI-~FG#;MnS*9!SvrVYpH=t08>g}AC`fb9rgY{kIc%L% zJt6cHQfp-Y)6M5u+Lm$Nv_{5Sl)dL{hp+#%2CNl<(a6fk=qYkemQ?^Sx}N|)p_hSv z^?yi*;cR8Rk$H^a_pk18RlV^_rmFzi0VT^&wN|-)sE?;zlqqV=0PCr*LItO4_SSUr z6EQ+xx1^?tD%EW0_58;cBNHOFlB<6;DW0)d12FaP-y5L9(%uwafNr47vP8kM) z?J+)oc`a?O)nqu*Cf$WIH{qzg99j20P&sN?ds;DQHFs@h2fb?PtLuchYiAo|tyOj0 zX>!GDSbM$gjk6j(XgwTor9q&-+JwPV`|J%)cYE|Pkk>VMR5Rc;<7;0re`h5; zm_r>_pE|W9dL_kpre}wDtfl;snWCoPa>|Og-#3#+Dr2%JFoygT3)s!6;+9jU!Z#rL zYyG)q4F(~`+r@Fi!s@F9M}6a5bh@FJED!;%b%x%1)Hjq-$ecxjZOq(Cia;&L1F6V! z_RvodcrL)BS?T!LC7Z7<;eUDg`{Gz=V1KBr0W~F9Dwv5SS_GsA-AvFnVu?uiJ~*U+ zas4Bau;>Q^Ex(PFuU}glU-1|3GR3bJXdErT@QIl1BM8=aO+$R_8*CV2W)tSd%W*=- zv^b@O$=(3j+rGFdoHK1JuYaioBTu4sitXrR1cOK@2)`{Q>3JS1=n!bEw#~8DalgV= zuCdCuzvS1W{b+-=PY!fx+!SbR`BJ(%0J!~S>^(skZ3gS|Y~aEDq?#yKAo^KNZ)77G zn?VpnarL>F&X;rm&kn$X*nvt{Wx?2doAt zcGinT9A(W^mg0A0E(}FcyoA_TxjZ$?$;Lts1xP*L-nO=ORvl60~#P2 zG&VcxPto{BlbHvAyD6UG>N2~ypC!sstc=^h!*zlEi@0MO1~|H`hN+M`9!QBHoW>0& z@ZMtc;49y`>mI#Z{ZGygx2|)$MUG0^k!}61?aN?j`CMg=cdY`j6g*~qTl54(L$DCs z!+ZZFZ+~9;7U~C)W&bKbxFD}-2=F8U7KeY4RE!|NFnuMLyM$zW4wMJ#3EpOEl_IQr z7!{9rvb4`WS7`G&@>UUP%<2g-$+XzI$Z_#kl5q8TyAVcOBxA&bSASe3sk;Evl?AwO zIw|;^?HXX@wVJd2OK9uPG}~xx7XHb2fE>H1r{4D~GDw2D5KObD0X{>+_Vc0T)(u@U z5(u2Gu3hpEdm7-%5lECw>c47qb;|T_Tfs#`D-rlmyd{3k_)PHM1fdTKVjrAw0cY^% zNk7;m8+lIXM+p%hPH6%-=_IZOp0ZR%Pya-SwjcNGs+{PDBw4&ezn1b5cpWXQE=8m` zexEOl(Jb11LE29avx+xd2yE2E2G1F-^gca@EMh0Dxi{vp(F|$XtSoB-!l)R6MRPp8 zEAvl_9q@XEYoOKoGAY<3=;q2J&jL&vV8m0uBoqj-8ETup&oycRcs>p(!%eM%tI7{I z@*Dy3|GxsV$K51PUgD`Sn|bUSd&3uBj?W3N)gIl!)7h(J3Z~9BGy)!3ZB{RdLX9z|wqf zdxFngripN8{p-+(U6+W0{;g-p6`|NWdN*KZ#(5m*Ed>no>!YuOnVY}ZWM7T#iwF6S~|@KzGBa?C37o;^VR!5#ndiZ&ZX;PUYt$ymTDU2Q_$Yo5_my0^^2TO7}TL2FYFt7cK%QZa{W}J~Hc4NXs>2Y2* zOA5YPNiwYCT&rBtl>(;M3^VprX@2slaa-rJoChq286>H5V1GF9s~M)Cb7IZETn0=DhX9;OQRrh&-+z)9hBo)^Nona`3t^ zkRqxnz7=CesjjXyl&*~{VnB)m-*YucQTE>sbl*F=)U4?Gg$Y?@OnbhX;FQcW^V&Eq z0Sisv{fr@FVxNEH1`t2fps&KA33dahWdv{#NZ-O4j!h?PvL9FkEsJ5g`C`$G2!zxF z#Xut9274&5$i#}iU_)15@(p5Slo_GbpV17Kq`N(J>A;>E)#Jlot{pn!02qq>JT?P0@7eO&BM&UR017ZG(pJ+GOz1`xe)7UAjAe|tB$sr`qa z@LquekN@HMvw*klZTkjL=4`*Y0M-b5(iOeb6FX95?Ydhd<)l+B9L?nFlrlyx;D_BHm(b-{_Px_=890t zHvN|YG2pq#71BX4{>fC{2loBVYFL&Tb}00JjZ7xPv`>m|>~nY8a(Tdr?BOSb82de8 z{9n@$b6z^;Nu{#9oPNo$_W6Q4G;;Z3*X1kRtcw>jj!3*qHdHbaff?H5k?yrV7)$-r z%tP$I{G#tZHRj3)E1O5-Mmwjca^fPDi}fHw$9EXvs{ripX1-LwTnaVux!pm8o3$ve z{&Y_Gts(V4oI*~%JfvR0)d|^hZKoI|=s8A^D~5G;TzA#KquB`qjjQX(x%4IM)>pdzRsAl)F+Eig39C46(1_4!ym7o_DYPTYIhd-D}VPGjm;M9OrQ!@%`M%)rniVA)uXqs>Vk$8T&6j z`a&xCv0VB#<+=Io5Pzj5!@Jb6!M|UnU#H$NKO<5n8JU)sQZRI559|vmS&3%{VY2kk z!K;DwwN|POR5~|$&6o}R-(jTFIH27{_j?%>Q6E(o`A_s18CMDfybBXFJ-1+}SpU`p z-ddnV2A@2GIzy8t&hPol6!34VQs^(UC}YjbH7g|fA!7&eqd;(`D)X&&G_eF zYAj&l@wRHiC-}N1cR^AB|cZ-fZmvT#i zpTVJhX*wUeT9pNXQy#)uVjgeF!M2uyk`L$ctkNyzzJ&Q(>Dp~GmACl{@p3Bvf_4Y- zrE4QYB?Ydv033CX@Xi;>$F+CCeppEyz=;Oxy=JqHBOXdB|JnF;yv4E2)#hji2Ygb& z;^i9vO%~x_ZHO7h+)Zk~7jjTcAJ1O=EyZOgu>Kg`d4uMB1ZKRuM}xf!0a6xChP&`0 zW6mr52bLUls8s&f`Hw;?WSNuzUZ#)drN|2!q>e8Hp?cQBsQAB8IR9XwDUty?B@}*0 zo`%XBT^%>wpt@*v@^RWHvk~Kc$Oq1Rzvr@ZI)Bz>{QYUA$f!`(@Gh=_QxVw`jGv1N zdPNY={w=>_@4#?zCqUlA8|198{4ndl=UH@t8@`pI5)fysJ?;G!wzkV*%OHSYr?aG- zKc%nlsNw@*PV@&@GoOZBe}x7AX+k!{A{A_f*r_vkCj%er=eOI7I-O*Vp%7z`dKmf0 zL_sKFBIQ5v+a(7e8R|oGfK=DNm%{Ie<9MOvTmR=9*T{)afczFWFk(}#a_kd!ZL8^C zPPu=y`50^wn)pjUv}-?%o1d!w8wzQ9424_`Wd2Vc7qU>d*0f$lgtpf==NK;{1CvR2W5H^waP68{0U)Bk^hhP(hQRc^Zwx<8t>)p=o+PqH(2oRos6d}AuY z5-ya=JOFeCvUBkyrnvo|8IgFQo&Rv=PGKD40IJ|K=%`(8`{{eGZbRy%&%^#(7X^GD zfe+*V2P|0=Jzl#ZZdEU_S6ZFD_b*hW`+ssG@z6p`k+&QZO2!Xae$VSO2YA$|xYa;F zuBQX&2M?5R+v)-Q$rs^z1Kv*(_Le<(OZ6h=t$MmfNHovJ)I+b8Se*S2zx1kU(N8_F zIT|ON(9Q0N>Q;T_^*ebTCw*7aV{YRg>N@|y0P&k=MgroBlBvR%fWp}0dgK?TYhykV z+x&l6+B<)$#Mi|O7ex**=K*&2r@Syt;|ujH5lqL6E0;OGk4p+)ejFsh_%q;H8Gi^I zdz-E#r~g!&hi*Fdae#~?fk?Xe^-pVe=c6C=yxowpb&Yc@8+(P ziSGF0yR{7z*Q8DJ-|)xyVAOA@VWJX7ky`0jvk>mH2??11ylxEn$#CdKIN5mb2exF# z27g#sOH;!w4}QpZ{gx#O^wJ+Us&Rw&WirGYpw+Ho z-p4imnv>f4e=;zvCjrdZ2L>r0>7VSc+{uQ|6Mq&_fxRdc@PiWYFTmJeMtCy&kH*0> z>XVs&N#WcHrqw~BO-V9ght^k}%EF`^l#7hs%mZc_AMJwTWkdCSI0ynAj10`1%wz{fQ>^gXHuPh%H9J)UXVAleo z|7@;rbT|?b`K^lIDM4sy0f9`PO8B+y-RJ#P$Flzo9vq`$9MXw`%Y4MT03c)272bdx-~npw=m z=Bx5|`u6km{>v%_w2JopVnis$nf|hH-AgI-Dhr3fqAYzDR)OkWKzF6e5~YCSHOEEh z4N{*s(9vj8N%`KW*bb;=YX{ZlvIwnCf*Ua%J~maY_@ks}!NOEH*079PIti1@?Z&{M zV-h1N!2X?W(+>zG$jWfo9>l!;QR3^p~Exiq);Q6iFICGW(8K?U5O3d7!CT zZ&xQDVBO5FA2%t|GmX_;X7KMH7Z=v<4ZT(3Tir{?tm|FD5wkV{w+XuZR*50DXMhou~0&G1^Lhpww;vJTC9K6%&RJ-x~ddYYMSk z-4=8juz4PZ2S9AL@D$v-aGF%qR)j-dCleG*S66~!)=z%K)}$ZDOrD{#{(VA38t~*) zBWso&*9O+E&GyU%w&N2>iyMMKFXR_T=uKOo?H7Ui{(ptBgA)}S~40jF`ST=z3u$vIWhL0RrpDj*DX^UbX?xl<#`CF3y) z3iF@1#@gqysOk1XE@of2Wg4N(ZqyPQBsc*A+~(%dEdvF9w_13db*9Bonu=D932_|F zaa0a#SO&1bQYyKD=em_buk0fe&Z<1+tk8QR>RhmQ(RF}!CtAbuy&1?>pcPozV*o(L zVEWbLCGr#3ih3$x8Lo{s$u7PiXu)d%?sB$L0HhbqM&8c5sBwsKSQp>i_J~rT+>XDR z3+Uk`d=TEzKgh&IvA?+aLIq$0|XK zj{A@Tdlq0^%<*DFCnX~mVuGM)@vptyI&wGZxM^0R<4De?gfS*uE#eBrZo+~32S6(G zXAD&YgQ!lvGHTtE+O=m|QS`-Cv#aMSO%R2x~{{roZ7g zFt?+6VqtIWB8fDxW2m-QLwEPoi#6{k-3kxTN^xBTQ%L!<@2BVi7Q%k=0r9{7u)TDd zD>n0fzKYN1X_+*M^eAqJ4m|C!*A-|0Ml`lMIA9zSAv>`qmCU=ByxlXkXHE&2eY6Gu zl(DC$HknOi9TlGf4paB$(AX+lR$EE~Kf0Y_ofkYDSLYqsyS=5tXLYR4_>(x#Y_fSE z_bK}0hvy^|a7aV*U|T{&i5Mva|$ZsxI6O3Er_NzRxx#rDW2 zL-l!G`f)3R6@=m*k@IH=2(CbJ-kQXCabJ*K8Q?a?Dn=9Ne*lez&Sb$}1GY-r*GKPsG5 z;#T>CKbQo6Slrfja3EmJ2#jtAG2N_rMX;c~_0}8=8dsn-(0-WM8j`j4(8AsjbB4!k z;qD?uLkl$4Ey3c~Wd&>j{O5lTO~vcY0-ygM+x1`3vj3Zpl3)+M>FD3Mua)bx`%Q|h z1Sapuf#V+k`Tr&i&WIUjOUX`gEWZ24dum&tLnA3{l3Yd)kN~k1z=a~g{!2soFDH&j zv?gvz`^@JfSbua4IK>=aSAYVjP9Sm9M4I4kfi#ezWr}AiawSa(v%sQlE*aT=0bssB z?X(dQup5CP{iJ)Wd+9`UtT?IST-?n)OW7_2@&j8lCi`_^L^7q)jNeIk`pf0dJ@z;t z`7@ZKWB1&u%wO7fr6e9vykc;5d)ru4g1`K;8VU!gXDw>Zk5PSb0x9%B z6NfAwIAcf)bgzPU8UZ( z00kZhJ)jcexgiimSFp@RZY_%#^naiEocl28!MkN}I5UaXPmy_ht^D@Ng`~+xl6)l% zJOwCw5WFvXZ<{Te9T0x~*q%0z#g)hE1SASNRAoY>jS$XT)>5x6E-D=Z2@f8E-xzPw&h6ndI>8*-%fbTAn-n#a5co zmL2?Pxw@k~7d9G)anWrlHUca#g0OJsLNSP!l*Rx8=jjfY|vsm<*mDtn-#3%c&|0)HGp~p zPWmV7Pv_Yr+|nh-^}Er16_I8cEzt^izNfY9Y`LV{gca~%T

+!pqcG<(=X@bsy#P9S ztD$~8x`DS7N}&xe_KN?=uYn4E>S9A@&}4o3o5vwu;eL$#d4vx9EN!p*(lyP8;62Zh zW+~TGZc&45LL%Vc@b{!tBQ9^Zyd(gfe;{Fv+0F{BmR$PM^7v}jdjZo5W}o?~fAF7{ zpACu6U)3YOKl-jEl_5taOjM=J4EByZS?)VXslum4SCa!s&GA{$4geI0?dNGvhU)Ce zhL>H(9lzRis?yzd?GI9?8%qk5FVTpz|F*p}8gs{V!bs`4g(RKb-9+yXl!rz&#UFVcnxJnjrDYx?{If_v0z(zuw%!xFj4X03yfZS?h?b55* z7g1-=&?`(>TyA>gx+^mQN3Xyb{XA0IFJ3Wz=Em@S6(z^sFU;H)K7;je860k5C^AbP zxM@&jtErZ8KeFWvgdJ+f8*k9~SP!gh6eUy=Ue+{0~jWAK0|^k5Il zv;q*tQs3TZdyrY2L0s3q!jxr+%ZzcN$r2Sv6HqP!5YGcWV#KCttJLRE(*tU%rcPqA z-v;hAdUbG7|D@PRlyItCIqQ0ii2Gc(WWf@EVrUGAn=S(2u4oLGlGQ659^0I8Dg`D` zJk7}CM&1(1gL0;j$~JvNEV6PU>sERc42CKw!_i^ibgrrRdwxu)R;7#GhA5pg&0u7M z^+q2KrG1?;JDOrRW(O>$X8xYBMRVgX#ulV(Sdz-4gw{!S0-f27Tq;EU;VT=gg`Mcy{Hv#U7D9SUhgL&tlueB6q`Nfgq>TwsSW8b4YatRH^{nO<{xH}8`AcVI3Mi1xLVFBEm%p6EnZC8xRuKUKj^;L^AdnWL6 zDf4=>J%XtT#K6tuM6H!y&Un@Jnvas90EO%t%DO8%E%qXRnDq;lqq(-?c^rhonEBd+ zM=%Z)t?_(M*s!ACn4Xz&Mwj)lSaKU#j_SkvAdGynC+p#uSW|xq>Wh0O|ALV$8ZS45 zLXVTZYu)MuO^pHF-^|hUtml2H0>ut)v;cp#ST5`vop_KGr6WKik?`rX=IgW`YiJyx zv=yZO-Gk@WW7;41%VUCD?}hzUKFnHap`|~s*i#kJ9n|Y@${eHC+r_r@)F}}QeiwyJ zUtJ=$iw1N`ErIRIr9`&e23&c=~C0sTvnP9m6 zT7+jWDT>~kbYHT}|K+VDFYZc5^tj8TyNuh$4D-`J#K$_`MW5oe=o&;Cm)}lb0lz<0 zyDWe#%;detf2xG_z#~$YS#M*e%lydiE=R+u-C%$VY`plv6&M(QQgQ8Z`Xu9L0!Zz< zIiJf%8Rk9R@>i2K=28W>BcGj$%Yt1;S9xi8uk1xGe}4YDptKm*lf{&R>%8f2T1$&Y zjgOG(^J80k?!Pnn-g9*d6-F35G$Gg5c+&qKB%+3O-S=wfs#ukI^Qg*DgB<{avgN_t zdM29K@YzUD-ado#a7RhOpd``0@R?=Nwa~e2yF!#=qmw~#H#>4Oj;p|re+~O(pMisW zQ8F>omVX5ou6}o>$rv;)m_~lkK*!npP2!s8#!q|Xe#^6=Z|@mDoTqD`cs=j-?T#`9 zX=s;N29H>K!OeoS_;WO8oH3L8;Bi0wGj3yR$Y-v7>exq=!1dZPp>f#v$l|Md!jtlG zwXqz;ortcoW|&B1)l*47b;+c00l=+#PdC{`*!yOJZZ^Mu0`Aq%`PtoZe4;^GS?2l3 z>Q{?Cc=1?)@cGj(l_uhfR~L4MX4W=o>O?gDMaI&4`QOP{1L)TL%kz$R02YI1Xj5Ch z#`%58B@=l*-*M>0Di9gg{E6t>NLcoVE++S{aHXI9jQ=GZ%MFn7-f#GekChh(=*1Jn za}NPB1N2F13H{xmK`}CRyS>@TG4HCKhu#(J;aWi$)9=_iy+VJ~R$ZfDi7l8Z=WzS= zP55fTzGx+OwSrZ_l5*3>>{)9@tv5syA3XtLd+6^)?$}mIZM)eeAaVrQRcM-9Gw2+nQI1by!13)ChfwKQY~d+g=sZSv59V`q>tg z!33G0H@gMQm8YKR?yKaRynizG8npFjuw&%g?dFBsMyZVZ$tD5&{hFRmdIPO{@^~DU z*wAaT&zDTn4zvdR?|a;Qd8&k4Vy8dpylEE71URUZW#${i>ad9FLD>N>8m9L;QIb z;JU~&@Vi88Ny$yws*-7UV9olN)Bx0AW1)tlF>n$G3si(Q8tEF89*hs137%SLvb?Lg zNI^%M$(rTfl=Y$6=0eYYz*O(LPmhn5fpxoCi)4lZNeVbx4~&Gj4Nt1QBb$J8CRs9s zlarsp?PxquQLf=Gite-7*r3 z_C;3g$QW-K&Dq>dM90s;XpOPAk$1>|O~aBf)0Ek>+B5g+xFA3NF*04-+gY>A9UYeK z!Lu@#F+<)svmvXrgkE5~ZSox}ehmx&dL*M)Mo2+iEEQ{Y3B^`BWBItv5zDUn#hlF_fY_L;o+3to%c7)Uw9a^e=QiaO@w-O^>3tSGlY4wA*5wPv|@( z6p<0(hw-X~8$wY9SZQ0u`?N+AoAQ67YmQGO1EAvsh_kFvN`81v=1Gib(=v7d^5T|6 zt>p)mg*xp$GAZBi)mRnmb&%Eu!0;Xh>)=uafaZCJ#srxr-h*xk#A4HEY)($3lU^d1 z1YXGf(zFc>j@EZrkI63_o#>Ll0hx?P*~x%?*XR)BO;CX|zc`&DUdt3`j!aMB+eUeZB@efJ!jYa^ z#vTGip(I70Ro-Aa^|(~*S0d<{$NSj9E>~CNElYjoJV9);B#cibA+vFP(Vmj-IlLe%LC%U>+n|lpN#mY+(Lo=1Ga z2xRuezWL4RvlDfJFI2ouYarsxU}80Ft%J&p!PSc9@2d+X2kP1nwt}f@rIc>m2JW1T z3j#(}wPAwi7D_yLl^zmH&rshUePVRvk2z$kj6i+cgkd)tMrwRlTU_~{F0{dJjVE=? zw#0{=E*C#XGZt4+Ufekr)dH)j{H@{LJ?0f*7rw|V?7+86#1zE1bSVqiBE@dPnI_*QHD50X z=3=Bgl1|MJ@5oyT*d|vfJdK(#xCQC9v6D{4*f*>Gu=@A_%e$}%4MsI7&N2Wp!bL7j zr&R&x<5kEwAa7ujdnEg+W8nP!l1LHSqc?#HeDe-&WviT~V*r-ID3*sD8+h4!ZX8j5 zk#YFex9W@DqcIQ0>}?*^a-9RInB zZ@&AVv(=3-vfGC817^RsyB36r?zRS3_F6e zTnl(8NndcN4M_?MWQoO&#VGIW7(BTvhONx9kMnBaZqNqmoNZOq(tJvJiaPC)4n-;u zO+?A1!ikUg3>`t-7ToiA8u?{RAE7*1SnzRLmp-9=cWn)sZTXPIKf$CBbF zq-O7|a_MHRFFAMy*PXgd0Ie*jL<6TW~G3wtThb0^Xrq0UF%G-*^JdfS+*zhnh;<~ zt<(|(*{QsxmHt15PgtW_uF}oDw5a^GeFJSR&*s+f6TZI}-&;lRMJT75WVOMNP}LLn zMD}n2DH|2;YIvoWaBQ9ibTknv9qfQEUVLUG@X#jm1PZ@?-+Yvyq#ha=YOegoQn_uj zSliKO8~*8sQ+Yivy-yP?nbT|2=9UDMocJGnQGHe8SiG<`+}t;pXI*2VUsqBUmRU~0 zWt-0YRYf4ZPm47kUvP3I{J*U-U^uky=A5_@W49={cnFQSf6y-Z)}8zITi23f;uJd; zwK)qS;8PDj#O7>}GaZWUQxi<)WInkE2%ly25D9(!VFF&CCr;rgkr2H6ST(oba}%gx zp>gtujt1NLUKj`J!kho7*P-=bMh-2@Oui!`nAuV&!p=?T`cWN{5UgqU@;|NznM!?& zg{tCze0Pi>GT{S^{ zO?n*2E`+U$kY_Ozb(a)~Zax}492tje-aq}6=2tKau!>m4_j0d!@Fw@I-WLP*hs`i@ zO{ax@Qypu*V&F7Ex&Zc~$(a!!%|CL9#A= zB~F<=Ow@Ufl1o{YaIRKWlETR88#GRpE$(rT5}YlYpv$z88P>9v$ztcjT|l0f;779jEtj zU8GLzK$dF>XNsX)9#?cG_}F+DPR^!0xZO}z>mlWhgtD%nfFc}2P&=l<9#nF#y`XTJ zsB63{NI$E&w$RZntFD5_O|p4HR&7{Ppzi|MYnb2>z2Dm8qO42=@W0tn04ol+r77}f zckQ2Lk=KU+BxmfIB+|UtjLB@B;QOf62`jP2iJ(SkKJ7p&SBA(8oGt?(p zQ9B2fg86gSnffc4-=WMb#bCBCO_i|ai{htc>qq5oyZZb<{?@=ZIUQtCX(tFF=N2@k_2R)8?|D=%&y56N=&TB`Fg zJE7Ucl650_OBnbmRs3xH3_vpX1ETO>W)`C+5|8FGa{9^%?@4y1_M*g)_M{oWG`{?0 z8WWg+$KKcp842$=+9*BIhvrE+ql%w&ZJpCk+2wyB2_3(Ez~Bc(1>U*s<eC5FKHX43ZSJfHqitz$Q z@zI{@b+EE65IOVzQ8%+000hO5ip+-hd)D&Kk`gRG@n=o{NdD$IQKGwayTGtgQyDEYaG7In!rD~_R9`%pnUlZCD>g^bcfs;gKzs4%Mks3 zg7w-N7^CpIWJ^5H{GT+za|Ty*~YwFFY!Pa5zrJ>d(A-AWRO0+fR*oGZzR&fqxg}YhK0qg`H!2l0teG#-rUCdi42Wg!RNN53b3g~NLrlK7Fxh6$NO`}}an(H=Pj&yA+^@2J! zq^Q2MuVyI8WYhUB9j(-RhnC8Www@r!KJKXe$p}OS&>*4v^2`11FB$UA^tG2>ynlen z%*tu21H2lXo1@Kz{^;U!neU$#aI754XhxWbpU=#%idv<&dvKRkk~7`f>CwhwWB@iN;ua<|1_r*HiD-|ziuCi4xt39r84Iu#njW< zsrxpj!o_h!rhUka%KqoPD;1sPrK%9JM(ySAnf4nm#$woBC^L%3SefhA>XW8G*X%XK z7jB3cl_AW_z?rYqHR^cOAMgn+mfpFjL0T;zB|iMMQX(VH?qTzGE5IN8b<2nX{sIQu zX7x#JunoY|taEuDbg<TYvZ>siwUhN_my@6Psk43;==_e=l2Aqc!fT`$H;fCHhc+| zpLS-fYo-$um)uLS`MmqJ`*%`+Q24{AmmxYWc}hKvmDM9FJ=|1{r`p|-2rG(k^4?Kq z_Jk#lD(zi!E8+e=S<_ogF7$;fNZp+x!oDepcP7$JjfJyl(lKNkUKedrSS8nORucCF zp_Wu~qnXc7KVv|XKSynzdBLWqV8kbn-gjNSI=fF(q&B;+LTE@_sP7UzjFC`#yXbH*->P^4I&UIB2R0qp66O5l zS}R>yi3m4PCWCLpR{&~Y1T9Bs62mVYv^0?qzU-$&_pR29+8R(=H1Uj7&wul!^|&V4 z#m@4qAlfQAsu}jiOLp;I-4YSO+7Ft$BAM{oIhZekmV0Or$hfwZ>A0QN;d&$HmBSqK z$lJmV?Yq?r(ilc`&gc!7K^zT-IcaYbZWb*obG1dz01X|_ampcX`V&ja9=66y-o|-> ziBKQQ>}!h~&`x5yLHdm9d-jWLfaLIQGR6?M);S zw`EFN!{@(|+_GL4G^X%O$&;g%jkWzS<*N^e@}tCK{2>Th0C=D9pN_IEo8a}hrb=}k!d61lX~OMp_L5%6fW7EVltuR zxKQ)0mro7|SjUJPv_ZJ}nd~M-zeUhRJtX3Lb3G$Dnt>)a{FEg_`sI~;I)aamKe=vC zVr5w+jZ+R}mrP!^P@Ue^ah|D`QoHC?{MEb$VaY*zl2rS2Xn5vzig;5wOodsg|0ji} zH(_o^7`|H+Dt3?)`A~Ob2e(*PI~k-X3_~F7g=ye5<%?GhcA2#%bbqZ z+yrS6g6lL;erI;xF9LJoXL*GtSYP2^>*&jVd{(zo)5U{Zi8@@+3{P-okFF9)bJ-wC z|Hy;PJC00IRcC3_(zN9}bFL#8>dxa5Zi1il2GPu!^ShA$&ORJJ-#{J;`qUa;I4@c; zcH2DgF=*NP#)#nAHclxQc9nf^bx-z%mf~E4_MeAyP!DTkq<}U!LZd)kn z-y1%GNsm%2bPqAk-9vVZtO>e#ftJ!=GqO-Z4gwcnchS&0vU2dD@&-b&8hagsLL&99NFzb4viS8F5 zke5T1W}piTJ`=&yzkP&@VxWu4A-NgmB!2oir(FZ`l#WjN45DVMQAS%}tJ+XZX+UOZ zaIf}>r7g2lZ$R%xFhja<1y<__xMI3#NP80%{N*`?mokgd;VY+n*~d|SJIkRES6k>7 zy3wMl4`w1KYe2fS>?dvaZXWYp?$oqWk3F47{Bkdx6NW8f``uzlZw`PPcsjp_zeik; z_5;ydtYtI{LewP#jj8C0RckL;YBW$VDMh~wQuC_hR-e#@Bv;DZIJ@k-jx~?hK1rrR z^37c`GLb5J7RLN=UTg**)6A#pzZ@c6Ov5v?(F+#llGqO`w+=!{ z7kt^q>(E2Fp*G?xAi9|o-c;hf+9MWwCr0$FAgiMYz9Pz92>=u~5_jNBVJ592g(=#T z+HZOZC!-4n6w!>`e%W^tZYvdQB%_&ra%ifGkRUdF=-ccrQR{nQfHGEOAYYxI9C3Oq z)ltP+(T+WvFInxK#G!VjdP!8%MwM5@iz7&5aWL}=O?1v`CHdyK>T`#3<5r0biqZG( zJtzGldYm&J#pz~#S%ooz><8aBj7W#pl8#Kg;kZCU=wTcL3GEwviYpu0jYirEt$k^i znlGF39)6_>D6(c(IKSA(yd)o%-wg;CT(x4?fD3^fCZv}jUITz!9)Ka3qvb>WI27hv z%P0JpC3YA*avwF5IIO>Ya>XG+Qe*u~>MaMYK)xzPDdhW7<4USWeI~ZJ)SID=E+noq znNzQL?uJy>c6gRIk7^Rdv~h!}*|jtx{e)MV_A)8ERK>F zHaDX;PrcZIJ!UGgr|_rahVWn;=-i|eKOb-*g>fw_tY;1$5*`4HUXXUhg*4vnCj)j@ zqsWTY|p4`v$+cm}aEll?|zG$afS=oD2b{3LwC|}hoHBjjcAv;rw3$o83o(I#du+c-t zULGXZbx<`dxP_1CR}ts>fAFF4}*PydF+iw(IRc7RVvjUgLN;ip6%6K zX~)}HAjZgpc&e}PcR=r!G}LpA3BG+qLjLsSft!~}<%L{AMvv327EM4IeG0~JDgqyl@h~-hx?T0En9x%Ms>bz zNd6P4AD9V>p3r8$=9Rd!IeV@IOe5CCc0@LKpA|M*k3+K^3Fmvl%_iXQOB^*Z6%{y` zF^qRvY$#~|KhE3S(H*HI4gq|5TaOD@uOC=n?fJ}wzN|4-NyVqg{=_Y$w|Z;} zs|MfwXo$nu5m5tq23HFU;qA(Y5^X29ECwO{5PN_%ixkM-M6ML}XvP{+q(xuH#nKsA zIag&UMi?$sgwx5DP6*lg&-h7gz6hI{4q8}UaH0TS>s*~i9r|vBpMX^G7ox(_2&ZJ)J%+lO(fgoC`Or6y{pp#-pLcOfSQS0#wJKC3m?5H%dW_J= z25>o~OLQ%`#UFeob!+*A5vlF;8j||Fn&V!_qF=8;%syT9a&9%vq1y__jNMFN1~Hro z?5{-A$bFb8OS^$}1oa#sR$O`esB21|!3G{^9f>Hf;_`_IpGH~Y7;s*_!o>s*RKGKV z!J^(JH#)s{xjwST0D!%?oTtuQ zo4}IIl_rXD`!(yk*}&RmZNLaqlmQCxbzRM-o9`h35cD)oVfaC}fyO-u+c^|SE1pc&7tI=ZW&YKu;Zq?jCzlo$i>Wgy$4*%gG}H|Fjyz*5c_di zst0-n=WHlO`)SKczsIaLgxM8xsXRe_?1-9w2r#6lp4f(xB1cDJG0NQD#VC7ZO5#wV ziJ=XxF!w-KMGf~@ky{?G%_E(N`ER8OdDTu{GTIpuS>f&6jm@XH4fnsg?v>oEd?c9r zF7>lGswC7$M)V3A-1jFxx7ow#x$CRX`%GFnkQiqeYBcQb-(U(%jfm zyp@!JP*?5l@tyncAd<`7ocdG0;@Es61OYpvcKGRaJZ-y zl3weuca(``cIbx_d62AC@+BcGntBScEu=CVM5i+5SEZGjr+09v#Yk~;(!Cg^?qh~BU0k|m9lNf z*T`*=o1h7Vf8RV4B~A(@e7>%}9U~VYC4xC@uYU>J65XzGlP(=ad>V@@er`+-;ErIv z-=zidA$Oo(+P&rHBTgAT%&}BhLNmc`9_3+jk33LH2HFE{gwmPzxp!gVFl>u5| zi*pWgcTcDVhQVy_GOnt0@ks~_mcb}wl_qkSFd(ItQ)kwn*^H}gUv1w!W~KaczC|{v z3+zlA$5wcBuE_o-;6`TJ*Q22e!jcCEZw;GW)on-tFFeBI5=-rL;Cr3gcdD*+Y7+*e zLEqb51O@`>zPL%r9jWo$@B>fT7^Z?}kCD0O;3JQCL45N3t?;Uxt#*#-VFa57A0L?) z(4+M3@u^S11~Gi7Mk~Z0+|~QR?wS~~0g8&Lu-CWi@$0tgfi2V~J9CtwEeRViLUdN^ z<1m)t$mh3xOeG$EX@aa4V!t0sDvKZ3q$2tb3ai^0EE_hLqOi^^4Ut>R%}@(KNW`V-)OrX!q{%$h)p*DbgS*!}E;I zR`1xvZ8+Jlc(@4d2eD5$y?gM~0` zd3`u2@-r#lW3DR}K23`SM80hhxLfV6ry{OHG{+GL(pmX!EC=IK^(cb5fSi#Px9f@% ztMgZ_g{M9E-4MtCnO&X8-Vl2)PblvSoi01-3p1%x@%Jn@^i>?K2DWoM*+fN(AFqXF z!CfyAqCX(364@TY0ybDW?Mg*Wo}%->x%Sm{LKeeE+-0hUck1oZzfkd17z^M3YPejd zEWFhd3nqOhh~baprW!_AcP~b9Ig|>beJn#lm72S6i=%Vp_vb9)k`^G&6AJbLvK$`$ z_x6R_iEp=X+YhZSxC}n`9DnB=If5o)*B`N1P$HggHLmX}4-?+u9!3Bjt#z8NzJK_} zey+^W+i#W5^_I@78$6pLZJ2YQjE3o8A|=c=!*`e?LWKJxY^g797}7Fkg4;zLoDTTi zaZ4on*1hOvR&!!M2Xh`*MdpFL-f^)1jcoF;Weak(QF}AX^cX;QV5z?Lvg|~gx!|U+S!uf9Pb#Amf+1@^LMGh?yoiDWUIl$Q5d&|5}uUV zXR21|+B2P(kLA#G&4Fla?73>;Su;<^tv11|ycFcs-sh(E6Qyu;SeORX-(V+du#6);`O=I8 z23jf0&>Fq&(ThrCoum<>Th5_GuZUS~J4UgMTLG5wRpQM6zxuZxY2UG2D95|w_gno^ zS^3+PfDQ+WsQ^-T;0Z=VRo}hm=3Xqv=5YIW<3xjF-ATEOS;fOgs0SgUE6z=)U0r{L@>kJi)68h%e9+#bhFn@)%(zF zSeIfoq)e`|9v#}dqC7lqHq2&{VK!cV9oNoQRi(&T7AEEx@z$nW*f`L+Bnl>6c{_mu zJbTVlUae}VSX*mN+U)*>xtpsaf2EwrmP2gbQK>I`_pUFm9w8|08Q*4+StI#WvI zlRk5Me*@0z$aPR>kcW~qefKsLcW=O*yj;>N$=I*gYK$m-{avpyI`aU@?yCHdG%YeV zp?_mFuJ2ee-24ZGf~L?%;KtcoHxhb_hi>BI%w}CS8@$k_1^42qt+g2CHD95cD8Wa$ zh*+C}G9pEi6h2&Uam_Piq>hoCw0I*YD6J~v3X7zyW?W9Y8prqu&X#N0X?rNegm^tO_idU&e24AJPr-)4bDE@1#LU zJ%KLam@Fmq<{ocV2wW6w=)&INU#Tc>K4>=q3ix*~Rfk5d7VM3)tfc8tPS%cgdGEW% zZ0lFusGi0SeKbHT{93p0RYrl6J36zhjI-5JOR)r*`Ao|~4Rcw&!HOX$T^H!oI>^)Q zGU@_#7C9!bwi_^ef0L{-@vi4oOM&%AOtpvt$^Ce8cOCV8TGvXa1aM-VvohznEt_Qy zj?o}9@VrJsUqrL1ZATEa=N|AtEu?#etf8JNbflu(C)REr_r3p=THU38k?}@ZD__X` zNdf0b5_cXiQ57{u@33h2d-ZXUlH95M?+u$hDzxgx;V09El?^M=_9{Mdr*d9g<#ej= ziw@?oE#*OdDgz9crM?gd+o5{*EJE6vfUn)|&U*xH8If_j#i(66uDoC&Jycw)$z#=; zo6}Z@#xejhQBjledKs#8rrH_fsyka=ETg~A4;m3g0~t#gj})L|K1uz<8Q%{Zw^hk7 z2y0CGqjmy_I*-!*6m?!D`A3<>3k&0om$Or@gro$P35MXx8?Wp%ck_#7qA0otr?I-J z@40xf-XfK3UC1x3d);IPLEt>ieF^;|zAYf&7u_Pf;R#34~WV z;!5xJtT1maDWiav;wbL%5!otZg$ZP>O?eL24A^}0npEN!qFnsgQyvu7<<#{9#$YS% zSmkx-*B>)exG1?j*05Gdj(Atv>{O?|S2CU&i@Jt<#e$=C5i8rh$fPzqa<;)t3gx~P zyV|<9+joS!^nUl(MRi<7F&{|PYM%~NjSuLW1hjAi_j_{%vfXcCo<07e3`4%o3YUlO zx5i!cUbPyN47C`r8XF^G_>^^oIG_chNB>IwpMI1&7$I7jTey;6A`=Bw?`sTWb4DC+ z+?7xb6Holo+Q%_`fgKI}2&4EB^aVWXScRkPry6Q}p~~iGNy21dCiOH6jX8^?p!1O6)N8Ao-@biBrd89Z~ggIGgxX06*{I z-Q%hkuO}svnQ9t&bElXP?7R#zj}a-r8&G?I-;$qjNN$*q2aY^W@si2{sg>-KpGLAV z-k81tye5swa=UO~I;+>aS1kI zSoe&csbz!}X`6^WzlTTy9+*`;QL#jA{syk5O~Bm86!*dFVoMbS$S*g(hy!v}`oMIKJ%@ry07yWO}pJ_ftu&Ef* z4h!vAcTU;(kfF4dcspJibAWYwks6dvD=}wwsvkt+$NG9k{_i3o`KhS8yUo($k!b(T zf(>6uH2c&ofZ!mx`JE{UA(Qw(eByQG3@5%06D8p!MmII9=udA+`S-t(%z860r&+I$ zSGl#_ZDLVZ5T5WQQ*k@$@M&OY!Vyk5jh-bS_+uINolOp%?}^2U-P@JT0Tz?h9Np2* zYp;$i8hWvnc4L8NxkW*LK<{4~kSvSW1AcY_KLbv{uA9gCC3zfxU-Uu!h*$%Ee)vZe z>3u_$hnUN|A2zwY#%mlC1LDk@ZUCr;CZ^ZBGJt~D{wGia!Ook%zm&}aF1vPbP;hY9 zUc+xIg?hHe3Oww(v2Qx@HYL2XV`(zhcW$}HNi0}#6HUI6_uE4(3!;ZLFx+=^Q zglZdc!zk|k0?JgrKEX?C&iH}s+Q#~sjSRHE@!_alNTt=`PWwg$`bftQ>Cn|hH3z~; zq#4JiC4IG75bsKr#2^zdM<1ZL0pC0x?xnV@ve)ikf6WxPu`o;HGvS&>QX*cl-enm? zft>0o$=o(pFL~*@9}&;=tMZ^4Xc4R;Y;v1i?b~o##VoZj_;sKFy-9ON0 ztK!}65)0q$n{*Sv`u<<+y>(QSYxwtzg@J&9D6JwXrSwn&(j}lGIiQ3z$j}ZbrGkKT zcMKs6!q6}XZUm%<7&@c|28PZd-e)kkd+*;`XPtMg^PaN~|G1X*)OFwYRo~CGGGUvb z0-Q5yUg&*?nbpra2bPe|e%6AmQw84O#>*}I3H!rJV!WN8@Zy@5&E06af4xsL%vwv4 zD(ghaOa{`HzT@(#6RR*{esE|+Q77nxG|Q8CG-QsXUi>uugrAK0Lc7k2(bfy;ewSKR z&xbiqJzPE}GcUWjA0iDip^^{9PciR0C?Q=0MoDVUs|h8EvEnG{<&LGNq=EoYzd zeYy`oH5N95S`t2`xB$JaFGPiG05$<(>oW&Q!$uF{pW|6-$Y7qkVfFi1hP{vRL7bHD zMfaqXJZ!uof;aJUBzxPjbmkN4n6;t$t^;V*wxd~Mdy^nY?6le^&vDSbnejo_*AQfa zS^H{Ijg@M~=KKC$aP#Ovpeva>2${s0D7L@pagbZMtd#KR?y|a4c3K)w7qAfC9y3QP zyM*5aNX|ezwvcDuhieJ$gD8V+nc$j{$dT-r*12L}ByHDYeVw>L)UZy}PAV51=uCRT zt7-oadjY;fD}xB^FKp9*r+CfcJoO!DN>`LgYvQvuJGeQ7-e&6}NNjbgk3Z)q&T?tZ zcPjps$UvLHt38u#Y2H?(jniNa$|51bV{vu7AhR^XqOPoH=3t9|J2r;J!fo%pj5a>z zbk#~hZ4Y=Pv^FLJs!yDdc&^A#;zB)LP5--4s+YiX+)hiih}(l9=B;IGHKEPbegi;5 zQ|;=**h1{I63}q^{&*KWugO3({mYv_K&mjd>dY{jvUDF#;I8Yt5YV!XsiKChb0FX5 z9c#7TAjGMloSU@O#_*YRSi6HkC9u@Q_ov9)4T@}cw?upK zcB{5itEXWs&v(=;3Rh`(yNJxNk=r|VoG@2l_=}h7M*0Ibh4lzh^SP+ymnmIuhN?p@ zP$uA-ViRrPy^%%v3!Q%Vy=NB4Q?I|$U0-VUbZA@!Yls5YbBCT*@aVU8IX;#Y&F9@M zEGucYi4xP~Q47Awz>oywc?n=BPH|Ffws$%QR!}0+R_I9ksu{NTfO>blzdrYUAJ$Nf zx34r0991(a2yf;s=6Q|ZURj++%9P_Ty0cfWT8buYr?ZHZQcl75w}g4sx)TkyvU%su zUIFOp{z4(uJ^tbBmW7@balAk*7a8PV@vXUP?($A1MCGdER&0<*j$)l^T(ut}lG6}$)PJB0e30VW0-wS;i#rR5@j`ayQ-=ai3cNv; zu186@NG6I`azBa`<{xX2*q!U&t=VjeO5ZQSR6MLuso5GI1r4UBMW{XG-PxZ5;Il6TNKmHfLwyWZ{b0%%<309eJ7|i@+K7Jh4337m}pLtz@Po!OcCR z0=k{We)%jCv{fP6`Su7B*s1q}r!LphJbaJF+<}H`jYZAdy}(dQY1SfWTRM8t9-e68aNUIei1NQ}}!$4h)K$se*Dn zlJzuRX&Nj>s^6Pg-RsjXeU!Y`q;|0`S;c$xG`t)tr+>bjk1$p{DeMcpQIXaq z688Z|0w8_F^Xh`mQv6|7!z6RaHsF{y0Gb5ah<~1%(=)NE2RoMi+xsLddRO{HS=tZQ zQ?06!L}h29B(%jVojSXcTqWluI%gng&TIE}W*C}F!CT4@ohmbNM)3Cj{)_W>fbQ%@ z_|)RTbbk^4Q`bg6&+G)Wf2&PvYBlM@ro_$my(X=x!P1c+)sxYNGa@(#&gS-3NU^2^ z&fQeqX=r9^20cmRxp%@$+w*Gs?sC+WWii8-c83JnbB(VgE_ko=UD3&^&o8>q0kNoZ zjeNNNMhfO{?pGG*7$ONbe?b-b}8Koj6}*#z=@+3F0bdGM{g4zh^R+&++dK zT&`BkWp38l!~k3qnk6}3K@es90bmBc%#^$n*jmAoKb!N_k$c${w8Q)ME2o#+ymHeu zr-3bJpm+>P(>A+uTeX4ElZMOv@|h9HQvYc}fTQd3QsTm=FP=G-pV0gq>PiIiP`x9oTDojTbcm;`_9SFe^?OSz4$q*+cJ3+63yCr2 zr~W*HT5SOJLr%x5v$=OL>8j)zl{!(+?7&-0`#Souy5wBPReP85*5-i7u!_Van+vWa zI15rcaqo{^_4`nWRqsDei0?9ln16UmKf3Re|2W*giv|w8=D%$nKvY729uwS9z#k|7 zX(r!fVDApSN+KfHzZ(Q0bokd#B5v&Q|N2L`k(9OxLcP~YuQ~@k!X1%Y_&k^3Ls1a} zY71SJY%p&y81Sq3CzcP(kz%?Ahs8*_>9+Czcy!^aL4TwRkdZ(Fl{xnWo^xkl-+Ha1 z<_?Q_yTzaVAUGhmkrBDS^`nZH(L6|SaXgmc@(we7E%)`xs*(xHvp-f8M*5ixjMH*a zkJhBSlJ10DU{ZWE8{3F}*fSpR_jc;e39RlbQ-4sns@PU2*!)X=95>yfV$lWVni6LB zW77G?e_HN$8)6bgG8T_DyKGNEd9OA7;S?O>RiL+)=XYt_|5i<^*%fqd@$vc(D^e0F zze&0$g{&}RzR@2RnaI9qiXOtDtFytFI28ZO?r}?C+Swmn&~N|Ur4FRN8M zPp3HvQRf_L!rCQW{HwpK!hoS0HV|G@tymqRM+!*ibN^we@-Q)TFgfxqH{Ya9FH-_8 z4l7*kvm@)>@eR`VXzb1dw&4xg<*lRB@1MR&*t$$Jc zWr1tXCWnC21ikYmAA4snd@tFMd^M7E~g-6U4 znSeqW)Kx7=we>GiRh*)94Fb*#&dB~L=UPKRfs`qQ1t1C_O}x3VbI<#~Z2UPy&Vqy~ zd)R|jttEZ;wrY$sP*eYbs)D}N-)1^ z)nCo1Kw8k@ztmTd1lZB}oqstY2}zVcy}SQ^i}wEMntfX?|JzO(d5>4NdCkv?X`!cq zP5n>(F?ei!dA%o}_PGTv$bBeceJUsXa_V+-j~`;Rxw_!)zwev!;?=D>TlJNN-Ju&a z`m;khN;1b%6g8cKTa~keOq??=hy$XVcWsACz;>;QU!iS#^`ofX=@Lk3Zw4|8yn7Qh zL{(OB|35B5M$nXiD{kJb1jdgvh^S;A?`q~=i22*rZ}slM4;@YIAu9hjN%4Q8@@wgl zI;2Pl`X91kjbEbSXNJs8X{6Z(r+x;64a%&q|9!uvn!Jb8BS%7N`Xcx36+=d>Nz-#s zsujEl7EZATBdz)6!*F-anhwG5O5rulFRSa{s`3>D%K!fGjaLD+CigHwz|*42yDOCl z+QATXEqKc~GXRzUN5sT!{juWgo&TWQ+fnGZXCq>t4KrT^^?)UAUwr&;=K$LyLNPeX zR0@`%%47DAY{U2SlIWN$dw+*{$DTQ3%Fi(4Z+rg_ynPeGJOaU~Kngfqr+>;;8i|-8 zIaMnFt^Zbv2B8pn&Gh4*I_7>zAbFjxItEd8ji5=^JoM^yuO<^(VgQM6V2w&{y0rqq z$5;53-zOfA1>~4SWv>r0?(pCKz?GFIfFM3$RM0Y+8_xGj=}c%al^SZ&H-Fj}_Dm%U z*E%LXLk&pBv(qKOKjj#w2)fOW7&`)ZfBHA@zF{XO&2OmMPPI~cw%Z$@V-5pQ`LGs| z;jm)MTBu-r*OWgueOQT88S^*SVs= z#y~_Q`|8gJUJ;5Mz9hDwufVQ^MU|r?+VU#yQz~kFTXvjwE2=l+4YY?v7^(@C;C0re z>^y`CexwAU#b$PJuwwJOup%@lcpb^rCaT>-iN>d|Gcem8eK`V~?W)?>P zr0ObXV^h0b3oQX^$Lz9IU^clEswz8Zs`jDg+=eja7Y*>jms;2~?zr5dW4bnX5@*r< zd_}B5q6sB9l6FDKEQ4#xceUaEfxDK!)iuBxz@7F=Q@f9@3)yHRBOtr>*U7=QO)CMo zs)*VKc5~HW)##N0EL`QFw>Nie>Y zX}vz&^&(v}PLYY@em8YR6`f21Axxi|Tl zVyDT8U>CtYan2o%?(}5~mZ8sWxhmSo&Za|HzQf{hZ@;|~q^dh)-Nb>zVSH=i4yf7D z{Jm_0;rK08M*JhD&e=C>xV*S0xAmPKV%F{k;hfX8c~>J>{8Zghg$?qh1BH1F@;hlv zym`aCMfIZF5yTHn~Mv8tgn!xY1U> z+*v)?xhyxq!eXjeaeehmCDtr0oqbdnXeY9@z^s)qHUqeP>wcg6+tRR&X7^_l~f-rR1AQUPdc%e$bgJ15UU|0QLt@F<>JD;;sp%69o1F8?{pS+g(Pi*cu^ z#QnX129_ZEUJ*_!kp3AlV+ zr>oA9IVcxLLD2_eFSk1narx0VelDbUP3~L%``3$^o#}y#0J!`xXB`b~Rs|Es_T819 z_{b?!!KzE>YAP`~wBKEk)-fmT-Vy#%UOhK$SS9N0G{MV3EW&`BqP zM6%-rctN)Qbu2=~{>g!f7*oK-vnFNpg_9l-cI33~v;eRISYx{Od%+B%KBAy%y9WaF zF{iJo0pJC%=h=+fx`(!tfArTOWDGD9GBntMQH}3&`NTpdJOQlT;sXGa5V;^JbNb?Z zH_#;Isek`1Yx4lK+VMt0MrgQ!Rr9AbsQGACqCPwslE;h=o@Qrh&(~ zY{jo^S@w7|6k2-JTB>P9Wpy}460VAdfKJ05? zN;92yECgUaKjY--{v}h#c4Z6@|DSjp+2{Ug(A@VNZz$VBtcpw&ldVZYP&2AxC&F*63bFP?OT#9{wr|!xG0+REM!eKzBOLr6A0#JYzURcKqAvF9 z=E-NH!fk4Q@bWnbJK$6cu$}0W;41Am=TrOT;8=6rP_+~@IZ*M)t&3c&B^<(Y@?tK? zzcewVrwV`vZhH=4q0P8oxo6jiy)Cm#l0Y^Yv@bvFw!;=(blBjN`)d%(ZDwtw<`eEH z)!q7hv1$XtG6R75pRUe+T}j0<*Q*kM`BHWD-+=i*y&`EDJ1*5ske^E@6|q%*a3I;R zVQD1?-dyK?vk9BNZdf%fV?5c&BQ<&x8#VaM{vGD1TYjpW=yW~y<^jO*J@ll;*nd0S z-UzGcJC-x$^@MLc5qa21ar+CudDXyaZkJ)#Vz%Z7oR2W9u*4YdXDg~o?9|kGxP;5f z^Vz$Q{fH8)Q9>j6eV?*(Tqnp%Qpd9HTQ+`vs8u4Us{uZ1gFR91+%`jBUpSej3R&QnvM;>9LNr-D0}}@zgeR)2BD%p=pq8`6a90yd-7Nc)>87!z+dQ47r?~s zhk+jENt-v)0%ZLDx+X3N15q1+ zKxt-buJHY+NLlx+JzywR1(K={a}@^tWRvLSH+u{3uOn)v_n$EdJ?tNf@Fsy3tX_lo z_uk4gNidBez(RgoK>z!^DZs1=KmXrEsegMmqW}9o=l|hvDUc76`aKBy+b?l zyk9rG*ch$M2C#LsUUr;*aTy~`xB;IZO$-{38gFXRpga~EhlKYMX|0qTZdKlK-RiC2 z&}tsvhysF->dwf$Y?srCjkVQ=!(cuvfXN4{F8F5<3)44oy_@*S6v)G<3?1Ro-e(zr z!*a4V_ub>jjRk^=WUx9^<4n@D^UuHo*~Dx%D;Q82P>@>HxTu4$33#5H5wW)BuwAGu*Y~4~O5oYYLQ9 zprC1-typzdO`sf0k}J`=xG6`coiCWaZUJB~ysU=@-pM|q?KQmaw8P1-i7(;wImyWe ziQTY8!S05LY}s-=6P`x6dN$tJ(>gHaOe*Wk1VhZbBt{X^7Anv8?wEac@}&C<8uQ);e*1 z1L)=W9H?>JqjyTDAG%EbG|2nOj6eMh5@D9PH9HpQI4kNO>?#zAQyJdRu@b!FF*R-eDTzz5X9nnPkrhOnj zcG^S94$*HKyBBjs6!MNflcSHVWcdxWP6mM1KU*lVAA%OXY?$)y$i#bGgTiP$*s@2g z4k==zeB%I~Zj0IhH&(n4_8I_RVNTp2ERuaU#I>HWvV5fDhiB$;9dBo%CMSMGHvmad zjQa+&b@L~!^=QFEU_F4%Pxw66O7RQ}7rE3-1+BR#cc}$ck~puRR0FFnyw+-1&F!OF zRFDTsWomDH7sgtkaMrtZ_9XPcpyJ>e6GrRVaarDiJeh?Wy-1CT{JY!syF036G@#+$ z^ByMIc~42Dyy^X&e_jfl(@E$?he1081&$P z>pk|hqIHeXqMSRN`;~-z2g9O(<9(w#R{lYjypzpY9MRkH>J||_Vnu-Ou%|lKJPHE^Fz>Qvd zhkRu&sWF7GwKP)qe2#?Ung@!*KX&&DctK4W#ZjwIMW_`X4r6HciZ%y6`>IQipQ;;k z-KpDt--YcTXj&h$mHixiettBbB#m4R2m@U{$Lxp1yng=r7%nmyLE4ZV*ebO`nX{Zo$mCb1^t&Pm0jcl!@Dge7;NbR`{I6E(R$GK}X z;kqGuV~Mh}DP1qD5jB~c&~SM#0!@DQi|Oi~fJAt}rD!t~-Lx*z0d<-ttQg54HB`2% zWL(cl$fV9o2XFP)8)uvxWB3K{^yuu--nxzS;<0$C#bvte6N8=&!h6G1U4k%QXL4|- zX3q3h?MFHFtS*Vi9AXfEaZ7Kx?EV^Ig*0;R{pg=W`8^|_ympS7TTQQLum85q7}xND zg+B71e4tRXq`Z}C&5h8@;m_D}Qd2!eV8v&?V>9JM+$gs)^Y2kgD8k-X&IRWBu~sB% zn)91FNIs(Kn4BO{hqb}U-TP#NR-k`qxH7U*RuF`{n;2>kqqH7OkDn#A# z7{utZ_!kCs9tp9-x#%j>=GZQGm9($51`2Anm4x1FxW`fr?H|#L>ID!imv7FXjpy*n z0Hk#KWEpyKP2c0H!kQq9@LB~xPT!~dVr8mtHJT4QT~*9U?s2Iman$&i=iibNehgmZ zos$k$U=p8xLpx-turNF4DCDN#+;LUxbLy#ynm>yBjLb4^iBWsQg4rqSoW6 zDGp_|7BexldpiPNiVxWwtYR@+*a3*6UT^yQdc85-!ENiNq0+E^7cj$q854GTM~M0i zUsUV5?~|90H?p`^btbRea(s}A%mSk6OKgDl^pcH(TyGS_ShWDxWX-JiKWYPs30(n@xzftu$a)z zzx*yQlEZ^FLU(d;!DE@8x}S=a^iGk5`HvLgg*{eSx1N8_Z@qZp)@3yk2C6k3J+Z;d zO!^la6V?Ap47DC8EzYttM)!MA2aeHn3$mcu=3f`CYOeQp1qv+If5yku90B@(F>9W# zvZRz}66(565x2()d=+Tzl$)F|t(RBcO2$Sc*)=G<=2=h*0gXBtbJ z2cuXnmBBPEFnmvGR%uDxzb>_!K6?KCIh>PRLsgO|cX3nu!~Xhe4wlc>5<&;tUwCr^ zzRBSkv}rIWQLp##mc*)i9!uL!YB~_z4@LMPwFE22Tg^v%ACr9wf%Xgvm;@e8>_(#3 zTANb3fr5WZ(wo(fTI7 zj>ZD6!sYnO)s2y6Je?3Ug{f#CKT%DX@Vnr+Uu_oj+NQKXxjcZsLI9Az*zk3T10aY7F^Z>o+>S;* zidxUhhm;DWtCJWEX1J}z`1`7M&M=o9xYHCu;f#5M0z)E^!#%85&q%zQ1{sfMi`zol z=GbJUSpbhQa-T>ls@&Luvm)plYrU=^a$#d`y)P!+6{vf2Q5eVsDfQgq4MYAw~ zkS;u|DK>-W>sj1h4yJJi4ol8#KM4{Qi!3Zqfuxut2Ki@Q@u`&he?L z{li6u@yBv`$Kkd6*)h{h4s)eG?~{Gx9zA)%C`t;>2w5FAO ziZZ9LY9Ml9NoyZ#dgIR0NZvg0qH#_CV5z1Q>?6b$n~wk#t{*DSd+b-(B222!LrpKr z*{uPD$9A&|eRyKWQ(7FZPlDf5dGcCbshbXU264nfp1RtRo)_Mm%y$aLSUi4ndfLi6g(ztRWPenDfLHUn zTaH*lK5XRJfBiEcd`3)BWvl_XWxEl$HS}rbU8^e@=-2;womOI*lG)ZVkdp+6unhn5HFsfYa*2Sg zmII{^15bqj*9{%jjSVxcFnDu1Y_tpC<<+sZ3RuFQG}%L>G>q{cqOY#;{9A9*koo;7 zgD5t7j#;+Vd8j!WO3ioS%s)8FHH1L~-TbWC3c#C~W%w=vVl`wVq|dnXUU+_k>`=Dq ze%A`Ip?f5rTk1#VP8GT43Jw$|iO{f?ynj3xqE6&|JY8q5I(0m=C<5Kwzs~*=m_!gJ@jaL9Fozv(v zc@SGj<2gs-&zaljG@4HWtEhse5!khG0rx?=OP65d4-&V3qRFp)pRuqt@QSwTVHiqETENDf zCyWt7$m)!^@=9ljO{X$&8g(_b1P9gUnT2RM|$)}oo#sNm+&C^I!*7+x#hPtK_ZAB;Vqz*X*e z(jDP{O~)i7e}<^Y_i*F-aZ4`!uYiJ>Xtmzny0q{qxnrbc31s}~><==AC8AkTmaimL z*GsA$)O0{zbKjeL$CWiJjN}P5_LHSxE2XcRqrG?7P0CZQH?vpJ)t>{Y`1#4`@N`;E zPMRD$!Z}t6>+j+@bLE6c>};@3lTjvVEMgX!doYKolw8|}7;JVw2Vm)PhnwCzg{{Lb z)+($t&>GSEZe}OJZcS~Z!`zwajIC4af(Otb_R*w%2(PYdQM)iO-M^J8zSsO#Ouz=f z|FWskLGM4+K2e%>RSLH3Aje_etoyoD#>0J&R>bY@Cssp>`B|ESuTDGJ4Aw7(!b;zTi@h9c+&VPuKh^1r=w6?|IEe-s=jR5|!K*G>AoZ5h=w@N1~7OhmvmXA9o|ZevKesxIoM| zlja57&q-8%#11^LyTONlWB*86SgJ`Z{^Z;Hh3)*9eLceKKQa3ZVI+#U1NU|2jC^V| z8uYL*EeeHcyqK^ULu0gZ)lGqphW^!Gm)Yah#2~&INF41e3?0RgFc-#5h zJ7jJxPw|@yz+8eciQA_o2*m?0L1^P4-#GjI{Fe_g>1Pi6mOGSZ&FW%-B0*;y^C^oe zs*#GTI;JuL;-_q)JF-_4S9r;_0>03}OZF8M(8Y)5L1 zYK@g)dJ0k6J;P0zVTMACc=k*45KF!(sLdGr?D~q;Rt|dP*gF+qO&^H=eNCs5e?Z&k zyl(E ziMLQc2D^?A7;Y~orh1-kl(kX2!D~1eH8_hFJoe3(2>D@(7-?lY-m;8WYC!Lt&*rE? zj_U@{G)?d6q@NK6B$Q{A%2hQtP3=be9U}xUDJWyL2pqrXeeEo#lRwyuZoQXmRQX@-+{O# zH}`BpX37yFp2Z(_i`=~Css?8PqQ8f1zA((7Lp!}m{-p;xb8CV3LC^J%#UHkSldiY$ zNA?cQj}eA@xZeD8xW^@(hqVi=ULEG5z`TGGqLBMl5V@GL&B3mb+3Y5iae@ao`R1}7 z+T0^hgyNap+7YbM6IuI>UOkK1LFzebj}u!hxpwAabgk9Ah9Kk9tK~Vl`2ry4sZZReiJC{saEe zK-LAr3;sbB4&E#a+@}*?H~_nagc>4%l<0-)kMlPAi}+*~Fc~Sp&Y@vQmaB^vv<-4g zm3J|gyK8QwdF;lXXJQBoZg$(hiVTOjNCUFdhXK6pQhUFkuu~;3B&{TLC+~;if{!9R zOVZR|y2@d~hk;f?XX6WpQ>=Wm=cPw16wOnB1`BoBMuEJPFfsj5gDWGYOHBsv#nqzI z28~GKO3a@|ydpS#+!`IF^;&zEYN|`ZDDU6$oZlB{Qe#?y=KaD=eIGCN5>QzIR+`Sp@)f*RT<~b8;b2D5GtfrP3B#IxeLgGWdd*+l zz?mZH%{e_dwew6LX5?ST+9oD8A*63AT%k5MV~&EvfOO6m)shAx+G+hr)wow_Kfb>T z`C@Z2Q@YvQR6=KJeI{dVdJ755K#(Rx>r z)Gu@@Q(d#|5JYT#^8-J>)j9*`_l=DeMiiv7`T~yAJcTd1sBnuJDMoZ7x&j??dR6WI)8#0sm?l5+1NWods%m416euWVsEeN zMX|9N7GmRPmM4slu~=8AUf~vbCJlGwwb|g+UW3vJe5^j}d^dp0(?G5m<3!%Yu-%yG zw^oR+5V7E1%+`FVefPVm{EFIFpHKsBaYwJ4LxL!ktjkiZL+d3g4P0gE0{#7TopeuL zS`YKzilTaR`%K0DL1&dsiOR})oK_n9Uo$Dq9VVB`>4S8~JaazsH*-mE%HXL!sBE^n zrbGWqH`uf9H`^G~e0&RGlr6l}Vf zj3lGVc~{z=#}WZW1q&Nc+IsOxzqPQf`Loh^d(|Va?XUmKC$>%rtELPrc@K`Ql$lW2 zy)L%s7U)gGnvrm!nX;ST&|e9(Fj7{3j0IXVxB!rIW$?>3O}VjFhJaIAn_T zh4ByhNnFhyepWoNE|$F*q+&&0*z@6rCc0aKIh!3D83SLBAj#*=%uo{7KLXL?Kh4ZK zMRVlEU6Qw-43tc-Xq_J_voZqfY8XBt2~kK1C&e)H*RitM6%v9>N3mO=jpC%cWiyof zt9GB{@Z~6hwy_4%Z0mQ)2Ls|a+V_Rp*5hsn-a(5S#g;a21SlBc3)a>P(xjm@>XRN$ znS0gVHK!5^fteAjLBGYQ%DNb1czGy`eBlX8QlZazjc$QT1C3|*YE;pYkP42{d&0PL zQfMYlI}Im_mo~>fn`tbTJ}`C_AVPhV+l*51X{e}~z5SKZo!GbhA!oE&(i2L6k=6mT z?uJ)x2!^>O%$pybSj3u?*Xn5`gfrSS#*9DU0bR8s1<^qp!|hgmFzYFUUPPcx!nsYw(>}%}sW;N%R8S z-o$~!2K7KDUOw^TdZAUat%U6W^9Q{aU@#D%HHfo1mPz{iLd^sgYEl4&r1i3Oe7#|% zflfTq0>?ePz`e9uW#dH8nI05CX=;5-;N}qfg?lXl2fTfr#qttow9bS2k3tjMjh=TW z)7)kDm5MTvPD4OP^^ujM7G;L8N^4rI5F4(f0@}p-DJA`mDQ((IM9qihD{@Pz@HbG8 zJu8c%QtP6Z8P*eLlZMtik)x#bFVwFSx^WOq0)_*>C5VE}&m>rVD3rf)>rGejtixq} zxP4*0EVLT26l=CO)TuM^coa~DBlyBjea!Sls^a8h)sL%fK%*zIan3O|m~mpd5*IlB zGi&(~y@yu}(cn0bheMioE#z~8i?LaYXawrsz0z7@j1!_OYe%3o$zE>aX_tqhca{1H zy|DSNq0GUIE^7uD-ftnynnrHRSXv6&eqZO&89mCBi-2}RrQ}GL-|r@HTt)Su3uW3>UeF2w3-xUR5$DjzV z5YwKosFx$H2xgcP4{;7p<>I9eecJWzGwJ2|wRBB6)nkL;zVqe+K8m2?F`ln!Tu}4507CaeF3B*RtYS8NOizjcItS zf8VYUKB}$(pAqq`bDakfJfPP$-fbeR6x4pCq=+dtj{#RX?eK)m)Lguq z`HL*c_`S1FE0vdwCCCLHMf+5ZTZyg}AYNN~cdIX4te|&FSP_NMu+N&wl(MxU<B zQ=-)vmmqX3{RhgS9;y%th6fT;jQ=oXFEwamXPY12+$A>dyW{faDI7gC*8SYKpHA6p z^+x5IwX5d~r0*FFT%`RrlQMxDjs#{lg@b0tb+LtdoaE&;^^p&(L{?5m(eRD-Kc3=& zxuU)Xi#=s6Rz6~XMR-J8QWX`PPVj!epV zrGuq$7g#(w$q&#MMNfFfP*_m;;jQ znEez3bRgkA#IPyBIr$Sje;bYI9SL+WeO0qN95zMb2(Vk|^hXpq&zRReuEoz`0Fn?^!gGfTRXA;~y9@1D>3W{_NOV>J`rY~nS~fn$|s!29xPFIkdLL&pNP@o3!j!9~B7nvnTFP3_C3ACam0=jA(Y{vLh zlJ-)81Q(RSoRIn+Ay%gO5dtp)A+W9r+exrnE?k^X6Gf{COFZ{y7HO-hxHU26+=p=u zD7HiSG%e8GqzsG&R2GyhW*C4HJOuDZ7Q@#O)E|e8N zk|1+B9z>b<$|S&DMo!`BORhZmD1qA_Sp?+2NSA8P^h63!Grgfy;-8dH!@m+Y3psz^ zCON8vYs2;ACClLsrDNpQIzGjWLWQxE=Nld%N+iQ@VarowXQYN67kg~1@yol>bFO?2 zNZJ6##xJw>)s^PkhKi>6kmjTOv97b?$?d%Pou{0{92pXbab^`0l-*IKaQtHzx+WDG`Kz{&F-L)#t|Ul9kSlU!Iu`in)3SH-6zj zfE%Bw($OVrNq+@@=-q(Ma2U^4xr68kAMU=Ib!cg~Jd2l7cWd^sYPZ#>G7yGoC+5?S z8I_tFsgsz=1aRpPp&uhiKMa`$L_|@Aj1iN&N*^C4=A6W?g!#LcG`FRhvPjgnsvaHqs+br704aE9T(31Infd!%Qzc!0i>qW zE0@ldViY!tXCUU_V_xwNQZ8!o5e^H`9(s_P^86?L>asP|`kl^LpIg6KDH5Ht?+yf82OsPAxIdD zR+<)luvvPrSafXQy;L}5iz7X!v1N+DNa&d{9Z`U_tStknmN55`)o7-n%wT(FeF4h( zwe|&)*okfpEV6gIW`8q~vv-;*Vv=&us_SSf-eYi2D!^W5*25%;R~0r3wYigyy=ays zt=sEy1?DU5j9+v0M@`jlrnuas=BuB@=E7f6+Bwn|Wa^+{K-Nw?%iXO4n~6i9u3QKbqUmmsz#jI^B}Khi#M4r8)@}br z1uc!};GQ5iUeF}Eec^#Uc;b3(7^&?_Rf<Rf~AcD;x|MGPlr4c~F^)FIh3|D8A-lGBhpwk=^=l zDaw=M=8G06GE2r|Qt$-Rtqf7eHTA>j+Y2Ig5PY!X=5i<3H!Ix>u6wqb+Bf;c9TZ3V5q4W+ZYwdCkXFIryhjpY!6Iwkyg5(~{*L!1q`b4dX$gC9b?7(z0eCuu^f^!Is^rX<^wK0`^@ zh1VRr7vy(DrV6p3{C75bYwI7di7YPcg#XYKtTtRDtAJ+m&<9ypR3SF6jYI{M-s^B*z7kfB>nK9p8!k_u#1!-}y$dN&sLOnKxDfht{SdtRtP{c&3 z6RP)_&!`mu7V+AR>OGk)4uBPCs-L?b$)DFlFN1bq&&#BcS0SWsO^z#vm|PkT0zS%m ztZK=*3}3D&N-wL<%tm;3apNRqR8ef3;G3_>HUavkI?FwrtmQR&L9f_1_&Xhwm~77T zE{Ju`I?-?PqaG*H#U!^PWnH~|_yj&8j9^^N;_$txD%GN9#{hqw_|oQ5?4xk;wtVT` zqL6wn-OZS03Sl|Ps6|}m07Bj^;wxa5WD{xELd)=A9VHT1s}Nmqu+iq5>qDt-iDOMc z7KOo`nfN3m7tTy64)=umqj9IWT2hyX>HNgd(Hgv)>81Sz_sn4KI?}?TL1}r}^iS6PVK!9!-HdGLc~Ss}0j5kZF5damY?|!RHYP zLx={&@tD6Ssc-Fwy2k2eIRvBuHf|ophO$UeC*jq3(IXYkdDWuUH2V6aN#EBJ$~1&a zPW|mct9yoMPGq&)AZMgyT)kHqrwvQsar*#Un=xaiYFUN$uqY@OjmyB`@}Bx8V<#n_z1 z!2B?rr)xRB#{_Q?5<1Q6sXN1(m-HLpts3r&qJA-U+5!z#bOSmjw+6ewg|0 zoV05Crm7au1im4wd#J)$e@7YjOmI)x_6Sg)AgG}-l1^JvJ9{%7S3cM6!n+*~1#BIV zntRa@ROq3*E#8T_9fkfja&Li)YU;c$K3S@B-`+O3@Cqbi=&^wg4c~wi81VLIwI?ZY z#mQN|&uBVKPN;H06N>v_3%BdbT_jQa60>~4HtL-v>b^H|X4i6)=WyV#vjQs}>%E(; zQI1`Eb@{>8X=8q(J06cV#H!}Sh0nV0+_7FhCyDOlw7|NNojJNzWa#ZJse)&Rs_JKY zBtq~D(4F$$_OgA6c5x$ob$UCPy3$&Pp7%AbS-=N%u5FlkPru^v) zk8EBe@cIs?Mek|yauS9lwvZ--a~gjRNG_5Jcw8Y?(%*TZ1QSu}0&CYqbO>?|C9`Ck zf4Hr-+)C-kEe$rS)T>mpSmqAYy+P|(W_TY=Y?I5;Tp)G!zHtxXutL4X1kxgm~yVkNN%;9t#Y zUW%7j)n2TJJS?t7$Vsb6cm!d5F&gih^TD2f4~UyxZZ~Pz;|=frn7yi>ES- z^6{UJTOZi(CDq;Gq(9y__LaTVo)TmBY5&~S5iWfBhjFWr%)O->;_y!b!@w5Y(dlZ2aK*OOYkdMFqBkCJjyS9ihSxE39X&&stnh=Y;1g50(!o~+3u#6i3 zuoD@g2sE3&&Z; z;<)PUsf20x5u$s?zbDFT{6 zs`0lf&$@6)Z$>`svyw|y{6;!d?2M7cPyX9WMGZh@^MqG-5@2?%A)ITWzkm{uVIa(% zdH<^O%Jf@qI+JSui@o=bYHC~G zh1uJ-6&1Ijs0i5TO+Y}pfJkovM7oNAbb$!c6-6vGsZv6d5IRbgZlxDVC_*SIC4^oA z7(%&o1yn%Y`+VbfzkA2HRYfX8g*rv51@RO_tMr6jh^#z``nm9WO}BzPF`CqmoD?j{<}S_w_NqhGiR2U6Gw+s{En`6pg~-z_f%)vDJWRI$tl0nDDD8GV$j&$ z<-bffq|kAc;Q85Wkv(R*PM;?x@X?#3eLV(?kos*)z{l$hJID`i4S6!CR3vX?bt}T{gf#WkX6R5Ot%k^5LEqr zy-^+6=xS-y%-=$^*xc!^ICrV#;b<;m^NtF^8MWw-Zo@UK3H2IEJ1T;Bo*L`QqS|s_ z7GTa!$zA=4@$ZayLI@_7I7~J0R%xqR+~V|>BDazI?tR&kEVEbM-hFsab!jXzyP%)8 zf|qKh#!m7HRqs+lh5@PAeag=BkP%}|s}S0vH}HunVB) zf64tn(IKKzcO&3iKL{<`f1FH|>_6=KuJ*t^)y`PY&uz#*3t!wYeq2#{^EiCE`PBz} zNk!1T`iCDZ{MN{~A)))udN?ft7oofdKc#}xpDuOmos{*ekU^D?EvTqD*>cs=-VFWF zkt4QS%#00w1T^=!|LPG@0Uy{}yn2N^NB!Un1Lf9v{Csba6m>i12XnOIu<8%z z{hT?^*yc-z6GCRQ3;Q{2CkZGnqU9`JH?*IriIa?W z3IZROn^vYA&5X**TL9mv*} z_yrD${AIL*KQL=VZ_>zp(MhR%&b-+qj?4-!IT{(6i}aNfXLEEd<9NMBJHDPE5)%s6 zoxkWlu+61YIWX=jH@~v$;I@v_pQXw_;Hink>4?Vg19s*kG?a)oZ(RB0A@dHr17LbJ z?)vII(*<~dS1zFfj=|G->e?Is%w(-TOv5a0?1tT{7D0&sbl$Cu$?vg@E18Q+6;YAe zoNzs&RHhw4za@5z>Xc^87-g>uR+)DhSH4(s8om(e4s@zkIj^yBU~%?c%B@+sFPJU( z9=-74{2`^R0*5j-1fas+Ei5(;nv=C_DF>$jx0fd4YFAugo&Ipm6<o{M`FcZAm@Npslb^seQ_Cuc)d4c|U})S@Gp;oW++)X>XSzUDZ4Gc7|qgp!-L7c&kgdy9Qi|(bE}Lv z7wq_pv)A#RMQDRG@hfc(a^s-!M&io95Q}FF6EOU7PspUYqbr^Dai0p~s;gE;+NVD^ zSx)2T>{p0wE;pVWsZ5E@SnMErsI4eG&!DQ$Ar!+_E`=NrJ)O+6bzzIkvZ!QOpg=tF z;~sNwr!iFXVrq`}$*QMhw*97>!j)$0K9-7u(b!p{%IT#~TROh3cn&TSJ+F>y%G^^u zGL8^uzT|5Mchz`x^YemXQ=P8aM_Li#lRci;kbK&h$ zmOolOml+`D7q}=quBa)uLtCgnAu_c7-o>eXL$`FI!dB75TLid8eAN9(md#w;5j5x$ zP-asS-cJ^vSC}a|sR=~p z6_3}@y(7lK^-_u+$3GPI%QX(W1A6co68F8+_B9Fh+awRG4G{=$)DTu@5F{Xb{aCLh zVVBB!T04)rBPZZF?HAB<1fP8@rX>r~C?3$g>yfS2_(a~RY)83V?RfNDepQMJ7$;Ds zy@b|}P;KiN06d?w(zmIFyV87(qoU14hsac*pJD(%_r!~GP*sqfp-(A2F6|m^@^pUE8>9jnL6``0o-SLJEhjcx&(rG`LVbaOm?l1rEvsL zr_0QJRgZ+q+n*A7;KOl+6P+3-cOg8Dyp|V*<93GA#?En%ya2OWZn;z}-f{Ct!}Sv- zm%JB=?X&}QJTciay?xw=MVK;WzASl6sBtdV+>*?R;bL(lpoxNnF`h%I7GFY_Xe)Up?*~APK1{6QxorP^Ji>~OF!@SAOhBSJ?kEMPCt7!>ULU7 zcZG|~9m;BauRq{JF6-h@(T;K~rs7VYOfi8neu$2yCey-=#+lrggg19rT_f+{C<|X&f6)UvrzLP9MB&?J36FL)2P}V+NWx<}Y4;P73(5R(Hq1vC-q%KO_u%q!C(V=-G@aZo zmOG@cz%N!@B1*q;?~IL*JZhGNvGa}>Wf*AJ+%{9xeqq?;%@Mbi+sz6Z`>u^Mo`%&( zw8xVixj>)jyP=tWaQ)#sk+(F9q>3rf^SG94IHAGgJ0wTx?oj79Da`KRVu>H3Ixa!1 zvYYk)b64~pUtrMuvnATS;xJj9aMwY~=`ltYKXv&o&pjy5&iT?J85ZJsC(Cix7by@k zyeZeLh_bk@y<9?M_+b$+K*L$L{OR9fvt><>K3F6ZiE>{a9xZJSeY734`-&~LD65*j ziF&_Bfq@GA|Ar3FUEb_D3D1^5x2p@2X;SNL9FV+3?B6DvC~zU4Nh2Z-DAw^1Kr7dArF2u1Oek<6ttkW*W{1)@ArLq+nDG+z~7uoB=GinwZ~lGAt>Bf*+cA% zkYs6kio3PIZyZ#y&`32vE4VfCJQ&4c-ali!T&!hR#>H5iPTOa@G9fRC+C{8Z@cP`Iq^vMh|i2@}5v+4*3{jC}EL`XoH{FP1Me&A8%Cj6$g1&Re{imgtG_ zpK#Zfu`t=9fV6aLq&R!Rb2(e7(~qdaJ~c8lSj z-Q%JqS!kJ#OS^52D^BjzctNn$!9=9*;!6JndWhGvTYgJQEaGKUui53FKq_h$%bCIc zK8$)H+Lg9rUfZ{z%zU*%Vk0S=;HtfGPc#3bWt166FFMQRC&`0a3SV&4Ra^w{rasRs<@WP?Yq+i`X z_k*GLrno>4nV~lX%z2`%H`ccxGbChR|4S>lw$^r34+icNB%;bF$My9h8q2cawynsV zePYW%_ZmfFN-?`oIv3{~<#<-xNY6!}KnE>F^_$h<3g3!1T<}_JjuNxMArYm42yP=5 z{MV&|Gid|YmsdxEMLU%t*XSKTKWX$@rTvwTuO{D#%eNln(lI4o)Y*%8rNDGwb+y}9 zl{_Krr*8s*^ShWw*+nL{Cux7RFQ>yiszG&`>x`+8VC`QW8#syAD$sAv%pE83L?ATB zj`RHL5$FC(_rBqJ(l05km* zB0CfQgYNR_+rl=}B0Y)dLf2`g(x;a zif?+8MnxP7Xi4AGaf@_LEsJIBMxJKl2RxW3!?X45led|!Uy(&cCd}18AZ6^I*=>8< zj!jRtT26i~hI5PM?|eX@xt%JdzS@KSD}nc}@Q{D;0Y51~FQ66Xv~oF7KAV2Ka4UAM zQQQKLAF+jQgmgzYJCywXHp&JW_|`uR>f%m1IfLC{$)5H`yz`!5dhU%9z&Br>0{G@T zO90_FTK`2=<=C0GJpHN90{_7no1KJ<1UJG?PyCEoXw|Nu5|5>o| zvx+h|qP1@sHvbPb^&>`fsewDKMwhl+V3CyyD)M?I&qOk~JKmHJq0=C4Ind_#eX_~7 zW@XI;%$NIOXEYZgpC{Yvs1lNQdLOa7U&#HyJN}Wi`@XKz37P|8{pJh=1D|khxad`D z`^)MOO;6715QuLM2CQj+7Cv;#ug?8x*a!S)j=TrDJt;k_^i;J=t20@&N zM=C}LhM&B-=Y}GzcbHM2g}?ohUWpxZ%0Feff9b(aXC`2JEN}FJ6tk!R^`C-U*}2+G zJ<>U#;p`)+Q{*IcWX9^Ck>nqPM*b<$e9*hf*iNZQ*B9`QekQC=Dp}pHq3Jy6PVw4w zh;J*z92Svn*JZDOPl+gy?*MSg8Hdu=!VHZfY&&PeSErKX+?Nws$uB-r$YAi-ofB2l zmvfJ|=0p~5pe`nV31B8Z07heiyP&J7#I-E@_k8ecaz6N%nzXSK|H7|e#rI@z5{LM4 z%3lc;lndT~r`9#2{&(KVL-L)6ZBU+i{r3Jym-jobGNd&}OajmX=m1s#ecA_rP%IUf z^a$IG7>(B`n_pHL>KbP$x*U(HFtCv)m32`nZ9tgo9*Mzb_!yURC~;X-G?Tft&+JP( zV>k411tj@aN4Q5_lJ+UG*oV>0uqvz6iSTS&Jy3lP*S$ahjgL_S&ukLq8_N)17-y!-F7aGeJn{-#tn_u9Oxd6!Y&mH%~E=37BdHlxU0xXBE^h>qk^?!Qz`3y zv%|Y{I8>ItzLSL*c0#$DU|Sqqk?&QT#?lDAq0@t~r;J{Kp5& z0jatbty%~o8?W= ztX(QNdk$RehY0GUm7^~zV1rwL*$Dn2H@h!OqiQ**8txsUW*6uYoR=vf=>mS z0Dd)0Pgqh;9yh{`JbJ=3;nbSi{R!@ezQKo)Cn`geZ#Irp2BS~9KXs5gPN(Osk-|WQ z(69O_3MJ-dN4&WMzW_RlSJyWW0FXCYbr%jaU7@wwy5+55GAuV>4--N}#6!m|4Qr2R z;rp^%kwvqgq(5DQXhSaKA8ja6U%i8tMHi;|)qqA0h{gr)&Ak24>|5}SKaU)M$OkAW z4$%jsgMTjF$fN?E^griR4dPuOoGUI2N4jRx7+u%XFM@bnx6QU9=eM(`j(e$F^kDYY$p3qM;oweYQH zY+*jhqfu;IB2AJMJ2KK4PI6@{&3}3rdAvL1u8jqEcoIO+VCl*fKm6fK&~%(7dv|?m z-{<{3GRvfUrKlCm(6)8~j2e0C0w{i!BZZpDnSE*5%!E3K3Yd%Cz`#yvqNUzx@Y0-7 z)`fX6yXr;jo^b3F|1eVv>pko!8pDf|51j9gVG^69RgPpky|EcB?F=2DLmYh%5a;Rn z9vcLG;rS{hdR=3|Dn8l($-POr1`(B(@ZAR)`p)D8YVSwH%1`tZ3o~O(&3#nW`iDH_ zq6GNu*zAVDU=TMycsdu>)h|g>j}Ox+&aYn|Vr-#qTEKQ4sv} zVtZZ{m<2HuKMa6v2<$-QJn#(>iZJYGwlYER+12w@E)WZ+dNz7Grk_J-yc%&!YGVFjk}FnZkJ~Er+YgNgRCHr6rz%ifPkj zrm5RjlJBbqiso3$v9*S)$cIV7Eh|b{F~L@rRzONQDUib8F12>rr}cUANRjpZe^82%X0}zt#8+{ebP^!VwZ;uc=sjY z=q_%K=PM4fij=SC4jLZeY+5L5#-)e4Sc*?j$W(QID zp;VEjzS%=OFe?Y#i^!nkY;X}56jPWFxa>M#&tu$Rp=;~(R^J{(R*rBR`8>>S+X&M! zLr2lK$cUT891WM3rgH*Dq?%Jxsvd5oU5Q6k&{1pxyl!V&R&Ps!O;j z;EuiDh}{)Xj)vL*Zm(z7wiwh?qxH;>`b0>u*1GFrg3>P9w{_3gR$DE<6a_4e(429b zX3p8y>i6romB!^zU#!YaZ9%l+7c4wxw>0}&rurF%QU}U;?VCOW&pBBgArhQ#3dzxW z_H5<0DOF8&WU!WTL$M-;T6j6au;Z(dtYVmH2ZL7V>DQ`b`TaBPvhN?{WOTRdL%01E z@b3%6#8xS$XLl(A7V2UY%BM#Cy6$VVQTC3)bhIb@6e)U7YUc<&L@Cv|M~xD^*6C0H z_-AS0lzWOsmZe$7`_&mrn}aQH)N7XI_s*Ndx?eaTGGLw^COovD`O`}_hRE5ZTD)>B z<0`wSaiQS)W&Hs+=l<6(iujdo6_*mO%6RdS zjL|)Q3QrvMsQ^S05U{YWny`~M^s1h@s#F}<@UGO=tc3`Zd$iPV`ey=WY72FeHgfjj zaz1%Fy3RThSHIEfgxVZ9!yacoTCM;se(BhtJ$68G?txT<-mB{m%Y;f17ysft8Hvv7 z(1PnuUu?sFRsI7oS$O+^)t?S*%D2%qZZoZo0D=3}XROSv`QReZpo8y^Y`%|gX_3sXW)dHqld zli&jd2G3c>-WiW}Vn)cR<*?KK{%gq%s{b>D`XdiOw--_4WyH7>*>Kfaoz26Pky6 zTwcG6tyRMruVsnP%RZp|4EuDMV^@G;{(Wofo2G8LVE)Ue0;3=|ocp*2r;^0*`F5jA zr?K-F^!1dEB;#k8>j6>!hpMf@?Ura|k>p+2`=e3K1A-5l`<~Cybx(dYDbonaVF)z% zdc6J){EZb`E7BxO!KFDx8uhsDQCmJ|el)OK`Tm6%-v;W<%EZhJC7N-_2dgtE6R$q# zDJ0`1YsunE*W`D5w~*Vqa835)9CKb)6og*JcsEbJIWt zD6as1{^!W*{|S-WEw#?YP$Mc>lFh8~1ESqkrQJEm4*d9ZL2{GO*lbx=Ym=?p zQW1|zw_BH1rtM=P^zLs5a7gs2njf_y3&n#!D}#0ghpRf$(q=dn*ea!S=^I;otiJ|3 zs)C)aYBg*H8%+Wbh{@X8#Tu1*laX=kHAtidGIuP_VE4a6Bbt+1-H9CD`{%F*`{Dy z)6V2E+QH=JA!+#CD40Kd09*Hgmb*BbvE2CTZk9)e9+&f zM2YzAH&4&4li363Y2*#jXp7{IWv?yOH+ymsR+NJLVeX=7nKg&4GRke2oJYUC>+LCT z;=8j6@IP;}Uf!L040Tat_EwXV?U~Fc8CXx{v1Wa|!|I|o?~%hWQnPm1$$6<}8&;8a z&2Lb=y0u~Hnrw)<(MAMerKc~;rar%2GWWWBJJgd4{ahAXX93x(Q$tmw+pe#Oibj@PiZ-wZ2gX>WNUwhvLDOo0;JekJiiPUSN?aCIB&K*s$$}iQT`5{^I zH{+qjurgXYNIis}NO5fMf4Ap{{{C;;9*5l|{eoM}Y?em~BDqE0^^6n<8W*P4*hcc( zC)-lCJB^mLNYuxcm3KaOWy#L0VM{l*?r1DY6|cr}=|>^TBg6&7%^N1V6;v4iUBY&o zo0~FoS)Rw|6bQS-O1-Mho$y@OdAyz$yQez#Sj8`Dg%JgqK^u z^WdGX*U~i43Xn+1B^xV=7_o9&s}g+9z39h+OtmRvc`a$#bCS2(+lI&d^hmc9S7zvPNY)84JjmIYtaItEKw5uKi>MHk+slDMUC zfnW!mG>74HQ_7k`@?|PeNs96MfvPtjCDz%YDEB@IS2aI4c5xb)jvt$g6?QEty=RL*(@MM; z)5zo@FA79PG>_ymnk#Tnbd zAH`m&c-z8-a9w^?m=q8ct}3%=lnM_9b|zlzpCl=CEV)6Mx$w2`?N8 z{Nrc2bxxv0elEmdYy_365-N(TqM0}{Su7{7gevlH%v{-im^0!&WX5Xm{)3^m2w4u$ zbifwBj{cO2e%K3Hz}qCB@4s-Hw9fFOIXsJf>ZgXorJUp9Mi$E^9h{eBMhlDIe|S&O zi5$26Xt2_|C67C=ti1O|@zSMCgn0jaVnMWEcjJ*ck~&K>76j)MtdKWaa~pT-u)P&_ z>;2EW(MEq${g*F~35X(S1TcbFqFb38wuFFH%CCB-GHJ`*sfCSpOK{#&YGNB%SYi6u zyt9W`jvb(P6fN&=bv4%y?^UL~21Rd>L-1DkX2>OSUpVxxxAxUjL)h-I&d#E_>6+rq zS+A28Uf3wA!ie^RV{M+r84crMNt{v77luFEvD%<2aCu)Xi;rDd3Fl%iT(B~?E{oW7df_>xTrSy^5!!dxs|hJ1Sz&M zKFh0b%m$+iJZ-w8X1etkS_g$}J~bCOmtDzEIwqoDh-?ifTc84xLBkk9YVVp&V1N4E zYlg{e^mcuR^+cEeSoc|IDdRIw>v^8vzMUf}bji@lFaxUv(_#vg)GVwV9+BIHVsOhA zu(1_sh>gj?cQ9F{_PW%bvl9gl8wll(?b8wt{k3~Pp#0q}*Czbho&GrPE!um3!{tB0+#9aIDBYI!|6@%v`yOvLNf2QEM1at)&yg)_Mx>JVSIYSc zF9lrQE`jZ9ApU1%A-uFI%;x+!8eu6Y(7|QSoNRfq;$sfN8RL8OD&~mAL&x(o(Rh=E zj@)(~x{Hz_5@rs=Pe%pq3au@i7J3SlRho2X+?-(;IXe=?W%#C@lHF|A7(@#M^$Uw0 z*A+Nrnr2#A6^IF8xGWiLJLfzXo-^1P7eprQc71t;l9qPwNCy^w*LAdG&^V}LqH11Z3I5pa-=kbQwau0TNB4SQ3Q^Se}#MAnB znY54n>M&C_**%9pEsDh+8&82rp5dsE?Y7M>-{g>U^Eha!(Hj5>4gz<-%HRt2|a0bn?>SwH%3&Bexzgr2Lj zOQv*%E20itm4_{h`&~h5*Q(+Uy6q5-p6T9inM=Y^M7PO5dgd|~yA?^T6?V;GzQSvc zPHGnfuxTuJtJlIr5Jlw&0sVf`YM}YYW+}aY!@1qa%l36L3ybvvnv3qMEK7+a(k;Ej zEsE^X^t~KE3B?+zcs1c8pB2_A4TY;}7JRjwZU^`-;u_b-i&gZexDbt*`HYeDiYzJY zE;Q%Yv>FzU#v|-j9D+Qj@1nlN%bRTmo`vh?xJBhbUheQvz0oR=QUC zODi}tYxW}lp7RZ6aU-OQn4N{6N5WOby{e-4$NZLiZ;6~Q_K;cX!;jIPYdutj#`B2r zQ73UyV?l2USo^|k8~>b~o3nS)F6QrX?Pfa^CpzZ}JAssinK%{ex3olntQ_HZl|9PT zUyn2u;#km3veeQOX%LtEYM9F1jZQVIvx6@W1JoNa=0slkpkRRe410nOJL1&sG~NJY zh${p5vfHG`u8izUix}guy0CxnxG}GsFlNDarxE)69(POMj%GR8l@xv;>2CAH?2Nww zWbu<{LXYon4G{~b_=dsXXI_08BS7d8HfPXMfo`c8C+$ID16w;snm!H`Vzw<=3)vJS z1V-l*RK3wV({=$n*~QaBQvaQC?!T53s@iQQqSEmt9q--u6Hc_9%j2quB!@m)2Y{-) zG)|dMfNZ$doqmyiAtQr5`zc#>G;bj89@w^4S*p(7Wjqg53ip7D5j zxw8yiHelp2J1G=efa!G8?0g`^iEH~pIUo1oC0_uiWK%=p|bzzJyU_RRM%cFd+OW3pW@Rtn^+jg zW5uTdXZdPLx{N9e`phoyr*qnytr?UQ+k9xid)`j_6}t3JgUSx`l(Q0AAj-MTH)A)k zd{%xXE~2gfW%5>v)2jc$Yt*N_P-LsSwBtYjG^7h@kACIoH|k5R-;s34?Z(>h{LR7Zh`D2QGXFdOnIXEo- z36x6QWue&RvSSCtA121iSsZ0iY4v31y|P<&GSWZnn8(#6UYDMA&jmHbw;B_N z$#(HqHVI|PA3*J!aIIrr-8Aw%@o~hkR4w1@)D2ILha5QSi#j&z-Jw5EADxZWAj|)+ ztVfz)-&y-Zis2qq+TjYJ;E|uSMmYTYl2SX>n3(+%{%SNZrbOTET__f_TlLE{_dA`K zD)-&~oHFnCTRp+tnr@Gx_0}4oTGC}!k&(q~9krAUdwTl*33byTG;N$XPm?}Ve))c+ z<{?XL*dZB;`_Fp#e=jwC4;MaUoyRB5w{k$sBBj2&$4hgE{Vp&pk7R60_M7|#PS@?i zZ*7u=m$ zdN+8cnrVh~JD6QfGn;+Pg>?aAPL8={V>{iKK24bq7u>L^>QNedJK8alz9&6S0l1!< z?sGq-k`Xu?%1X-F1jC1Dg^6^XP}TPdbD9qo*jT+^pSb>X5O4AEMSNnrcrHJ_9Q31Sl+94w>e=K_fVhF>lU{~L+n|} zB+gC-CzMZZ3x?!gwT zj9=RZYIP*0j)V0v)fBXV{9d4dkntwu|^m;C)h^xm>pjVOh=+&yI8avV1^0FkwbcyY`g18c< z7PI-LKLksB+De_&ywQ*zRc_RyP;W@3zw6P%ID0Y4I&W8hGP?N*0rA}{DVbYvq!bhv zzJiIc)m`Bc>2`&uY0?;aiW=T{kOvpZyQ(cblcv2E0;?GY5i!2-@e?B4C&FyhO~7I1q)+A z#-ClU7#qIY$B`jkth9vqo|L@4+_+F=l5K3I%h5cXBVdLqkyY{NFzQueL?`QuG;~~8 zX^$I!a!?M6ouiy++r=v>S=tA*e=Z{Kk|5hkzEZkg#G@RJ(vY1T$?%qOB>UyiIH2@6MUUDK6SOVY$ zg1#QEn#tu%X}|Okjfa1Z*}+P+xpwQomr3dSI38+hsEziC0KwJf{5?E}6C@u*tv*i3Ahsegw-P0;fsh67G5ZKoa!+<`7uc>xGAdg^cIew1u+f5 zs~mU!;=8#;o47hUqbHDFv&|)CMqa$vN(!8^%B*v2t(JP_J9`TT-;NFB%a-KFaa-lO zF|j>n&M0^NN~eg$PY83BM{&!41Oq05jAz*qZ*)pG->!i z2rJ}DERN38f05g~QMv>3f43{4>dwY0QS?39TpowxmkZGMRGPRi%@l>_$69rwyZls` zd^9t!M7!a)xDjeop!SdqXNKNy)r31c=4(@&mN$ldbbdW0J!dZRw^D)iQrn z5-O=@k87?vJ>^6i;7^eDUGwbxQ(yFg9!ZQ_o&Xup@8(>$v5CDR@THN6Vda#;pNDid z7spQxbkkcIg7%JnCua*`e(iL96H|=SLWO-x6L7dGYV=~|8!8dbN*VoA=A9ks*^F#4 zuA^mbc}4g$ZVt7{<`Ef4PaNBpvax$p(m(HHS#63*h8DqIhZ;(8zE3DHe8_vsv(~}$ zBGDtXY8DnuoX_IkrjVJf6+Tb;_1=QkgY}1m#NcygSwvsMNU2)k9=@krcrUUqyiEUv zG}WfVVRdpEx;+zpx48fCz z%XvJ!_*&%oo!Yu{^l#LwE~O6jWikJ`h8|(zj^OClHl;oXk-)xS0K)=&1MO1G4hlsU z*1aG>0QHr9-xO_bgv5*Grygb|O!DCMAlLHS6?m4-`Ykp0AdaQZh8>DfjmU|K(=FiF zl8sZcj1HCXoDE*;q*Q~XCsgOjEG71l`5R(wiU?151wRd;{u8%|#%2KQ=kEO5hOip;=PekxgYUkDRLP z|Dq1PLEw`kxQ$g!P(oOJfl$?jKRwhfJ&& zoox&8EendW5vs;L#Yd7*#=Y|YC%$iGC>l3b6dgzw4zSctC!`p2!pt#9)Sk)%WWH)O zpj;w3j7fw)irXMvtO(Tz`=cQ<=gJrq{c7GP1o-}kTg#X0o&sxvLR;S4uRHW#!&Y^M zoLME?KQtR;|I9iX)em@Zr`mx03ea|Y<;E?uuB^Pww7wz?il!15EMtCZzyem#Fopy8 zbV`C0PzdM^SI_OZXXVt(^v!GOJKOfZ;pw&0uH!2tFUAZEb4O$9h|b+Jwlp4HU|Xod z7Iq39Na4>sM;q@`=52Gop!(LHyR^jLDS?{e6k#6UL&kOW3+@_4;v zSRQYMj6-bc@`KF9W-F;(Om)2M!D5X-e!r$p6LkqiNd+M7fQQ#OaC_iOnzmVWsb~A{ zH*v5}=}XWWG)bx~`l%8Q?BM$%fKSWMF@~<%h8n(y`_nkHQF_K~fmG$P6K=rFl!FaN zuS}R3ax==-yi78pMylr=eRiUz&j6Ao_PDrJD@B$gv54t&V<`05aW~!H?Z7@IJmEzh zrPLRhbBBh%7yvhK98aZ~nGM;}Q=}}T5RK4*S&&~C;N?D0HX+LwK*}PE6!2L%Aj6AV z#6`~U#@DZ}D0BzN8dTc%)_MO%o_sGUvfSiN4XxTXSm?Zb`jX3}KQ?eru@l-Y1es%dU-MOW@LcQ1Ql*6}J!-$JoeFi_(X%9#WC&bFJ)s?5u zhCSj&w9K#$yqEw_4H+mqoF>+YrMk#E z?2!U(Sie2DhRA!hu%@QOCZuu4_3e1aChqDfMFkLRG$S=rrq5mgkYL~rfKWN3ZsC1K z2SF74yI1mho%QXNzMC@$D}>LRfd@rA^0d3pzV;$~0ab?N?6hjcdJ!Z**(&kbT1|&i z7=SGxE{JPN2XCgnOhByIJqO@C%(OT6=zLCmfnh3Vm3+c4gpz&P+fY0ZHvM|O!rh%4 zcupiIWAe;7@K@n|S#7|jKZ@g=Xiy)p=N@<>96)W@yHvQsEFaf#y>?bkqxt)5abk&% zY90deUb8im?fhE0jYzStAU_8XWKW}d<<#PC?dGbX&bIcqc`i}`I*LGsz?x@Vvi`o8yMIg0xVxn zsgO)qRS`gYZsB9QnF39;2+&#QTuXz@7o_x}t%bay5JE8UqA?W)ienQ3YsMa~JsTSb z;KiX-UU`shN6l%1)J{KGt{=d+yL>#_NvBgzq)zjuv2Sb=WnY%Q1i&T17cZvX^enV$ zL>E*F9eF%4%MF1mobtmiq0_90U=xB)>{kRN&husfE&_}ZDi)y8OadEQ8=MOC47l!t zE0;Mgf!twZeze+AbeK+T=t93x)kL%^uyWAmO`nwn?gc^#z^EL9aE<9vaKs;-%c$}k;}f|3Wkj%T##urIuYGo;`&YWrrBfG^4^4gBp_0|SlVO;HEO z)vgMXxt>7&r@t+loHrr|ND0p;Tp71(pVF4NpzX7zCilet4v3KHrNWNlYEe`vNQ(=xA}Y zcOv0Pl?Ts%wCSswtifvPVi!KJ6l)6Y;-LfLZ9eS4-hq>cFnm$AC$GHA;ykc7(5?rf z**fM|`jDNM ztt9nSP9$c+eS(QJN~yL(0hj}Z6>$JBp!dv1D{$%5Lc%2*cs!y1N#qU)Jg>rnBULL- z_kyd6LaUt2J_pW?Z^gIkH&u@@$^z#`UI<9T?W}tHqidgFj|eoB-N1luR2GP5r*W>5>2qBbDyTcx(Hf#lHzLuD|;{1xzM2bS)@z&V?W zGP+)2lJ{D|yA%PBzfK^%dCqagq7hnKl zs*xz9CN)m69g7!(q-z}fq-##{Q`Pc$9u$XQ(dw_0HO53VH@-jt*j-ktoTKw6Kw5@6 z!1ssL;$N6jD!k?wHgSxID)7uSwSvm3hHYJ{jBR~mb%G(GQdC0LpTE4hA})GMKS&%K zlYMhnTx5}{eo29C12NKeCt+Qe@^q>BNWpW^R|wT}Rdzj#637C3SPOW=rWh{*n5(Q`c&lxW7q63G`Ok}f=CTXjo3-9TjN^GINw35rHBoGLWoTFq{41{|z; z0@O5s>;cc_GU~<e*B?_i(_1J(i**ws8H(!?Dcxu9nNkfIKd?r^>)yAnQhl;ri% z)BOQ(vR0i3#vwfRAS{KEzg4M+i3d8+T0=bNc@E&bvJ8N{LRZjEi~ zu(p`q=Oy^Wy>>Sv+OxO1lZFNpRq`1{a~)n5bsFIhs?D}`!xi3SWwv6QbgVv_c~y{m zz1@hBve+YCm7sz8(aQ5gU~_X2D}(Rx$Cj4raAS-Tq>slv96WRtA>q!(NiNLw%|Tf! z5Vxh)A~hL8KDf9~axKWDK_PY(a%~HmBLNoB3P-aM3zA&9nSc`L8Z|)+2MD!ROC%NY zijZNAbWsnVnu#-T>og)cmq@|Bn-R9av3l=LW+d1zv1*5|nNMCNlloF=8(mDb!f5A! zZ;5GBCS1XHELlZO`O+_&e^iUBdkn_=8MxQ-;mtKosR{5}+}m1n!6+w_`QELNsRO>x z*9nO-av82JjJr=(=4^9@$WT=g6ZM2^#7`ulIHF$A^x1J2zvHHdBc{LJjvaXEo5pD( z&oH5)$TGHvJ(LrNfPStB53;;6rqwA5bP<B zc$M)nCPCJ1^`vr2V-p~}sZAiUhH{c1;qxO`s{o(}&TgaXLLOdgz(h-X-CYvuArCry zR}_8m6C)Iy+@jw5R^INdP_Dy7iL&oUeAm^9W9E+K`qDN|D@QM5jwsll$?gt`hqE{N zws1xc%ap*vBndcP3q8X53=l4Jh9pA=L6-(9lj(FEQkKh~X5o+3E&$Qgtwf4R#M63o z$!Noq0VF^(lt2{ey@@Thmq#K6nSlV%uZr|5|G=7 z;QA99*rd=ZUTDp1t{Iv?FQ^Z{G>-0~^!04688^W943EdI8soO!qhI||I-sG&b#Q2G zdVetz6AJR4i#My<8kuyr^rjWd>51$)c=-;YApd&Hd%(R+sSF~kNQtDh0Y$J*C?DSr zg_VzDA7}%lWQsl$%E{<11CVJe*YPlJJ1@!ZFCiM>U#xK}vvj#qAY(OP--w&~qim>; zAuW*~_#zOrxZNdN-vUE`_i~3oWwaUi_F%8elP~HjXp`cKzhb4XF1GZP&Hw5sKXHXL^HOr%`c66@sp=VDf8JepY@)e&_Ov zBfnr+*9E&ov)raxk97P!?Bq=)RBVMwL!{G%6+1Ed6I_V<3so{8HOB}8vPrJX-35y{ zApUncd$-L^&DClbJrwFbR;HbPb z)8q=_X&uvahzcMA9i}DS_B#~7Kkd=9D1+fo9q=!Hs#WW72ZVC$3tLQ60Ft8`BUe6l zAi-a`p124^xfw_~aFAY5FWui?t^wx|HW1U)x5YJlXP|o6z-)crViFwP6A#>~XPOQ8 zt;OA@M|-0a#AZ7S#}?Z_M2^$=cFGV~ua!osWk@m?TF@L=as|3yFV;xp1eg_owBN0k z?-jUgWDxiKVU30UEsfs&6K9=tN=#ed)Zx8JU!zs;4xmqym$Kld6Jd?_*YO5vv?Eni zp+8Hq79q(ZW%{EtInj7eE(Ihs2JJmzA{KVQrV2~8761wGyw4W^f~R16BYIKDRrtkq z(uI->ae25_W6wp7xtiI5x31&%Y)~=+w^&i$dL*qwC!0PPIEd1K<~R~agIk^4#f1r<%ICSM4)It32*9<0!E!2C|_A5oLmu; zH|>}QpDO~fa8j9+Pp~P>7{qJMeK;kMK{>WGJ)=er3L(@D6Y*HUs(oB>9BV!XU=b8x zK@}+lHzOvHbf5-}29R=w^9pOpGCos}U4Cu#8YH&>YGke&GG`hm z1#}(#IN-}-{pE9Vx2KR{e>_QrT#Gck(Zw}0S7<$@j1}#5E&=yKs@fZ33kHNR+YW<@ zWRvn*RevOYd8#mOH(Ii2LA$6N+GMc+H;`f~T=}FxDs=WbdI}tpsW2|2e#HrT@!7E2 zya`JpD+tyaWy#(TctYwJ;OZ;wp^!E*fyPh-@Ys7DO6n!{DCN+ogDkZcffojqy#V7d zPsUb!wxl>uJEJTB*E_PgJMCU*_1_>;-z&@h^^uwJdV=mh8L9c~bufLN5 zGtTrcKq-(kM~zj(>jNFz0mNa&W?)#2@ew*r`c_uFP^tim2mQ$jl?6pp;+$5qi9ryAfP29heCMrd9995HUkq0&+BvrYeHO1}Fu=cY3H5_j89Gh9Rl z(V$MRz0KOC+%~^vXY|vU7lTfqtDA&vQ!bYk;Jv~>qoQ8{;op8DbB17rg&Y2@>!_jK zB`&rHjcqm-`M@|`o6VKU;fG2kVA08Ii+=bQk`ozHEL4K_n!WaQD+<_M``5QZz_RuzK_T&0kJ0{6PzBUgT58nX z?uykrGkg{HkkV;vkLBc+!P8JVmcAtn${)0X{6TRJ(aphwN#IIisOf&$)y}E3B`wG* zww^VUn5rE(CjPadH*6#@5u{R91=}U8iuda7=2#tY{Y{crOIGi8# zS$<_!^K+rb9_{9A^OWgo&N||1>3D5@Pg2FRYl9UwDjHOL1IF(-s68vO%=V!b)yk$r zyP7oKoq9A^TKk)mfeJESZr@U6Yn~M>Qnz3xTWm^HYQYM~PLsf+>07Id1{J>$@a1VS z|M2A~yK?kyG=IC7+BlB&>V8>BuNnjU%6%=5^0%&*OrU}VI#vf{w^xP-?6%=wbI=Yf zJWCk@*+jBTf(lhyh8BDAE1-DqW`n`m)d{QpulBw>tjV)|+qPP2(PE2=ihv6d5zrzl ztU6Ja0~{D3wSbf@5I_h?s#YnYg0ch*R9UhNJFFz)LO?+F2q7XP1PBoVfg~jP?k9l7 z+Tr{2{T<(T9H0E*p@t;Sb3gZejq^OO>jIB3-RtijVY25o9Kc`uc667|+;7p}eIrC? zDndIA;^WdGJ>fx*R)}5_qc?YyEK#<{bOFrhJ}(0ha|^R3hEpcKC&490EKR=S{MG{i z-vp7Ky|c&Rcel*FI!I#IBfffWjDsGnXoqYP*7tIb+FoEz0f<@cRV#zzCWwTtVTNU1 zyB-)_eKtA)GV#CNPtX# zA5Lc)t54^xoBeA4H>cJr{do^VY^W#jBa!pD#<}N+K&FT*8NrpvR}6sKs7C9BzkNF| z!hz^%$CMfEiveDIT&y>XS#R)cF3nP55FrD3D!_h0w66HJ>cp@mSaov~pd>t13?`}e zbGL&yGJQ-Jf$UeI5tk();C5z)<_?v2xPbIA5Ii$wZ2SDlSMT;Zc&1@FBSMG=!a%Ut zv()nYPgws(Jw6jh5cS7|K&i|QHDD;jhi8snuJR&|H>n(|^swGCcNr;w^t_5uxSU>9 zF`#609`^p4TNbFdDML>hc&D-|z#47c{2gQ({$!B=xEyux=XZ7(NO5PVD)HLiQ3tM0 z-UP}PkM-FB7IwZ{2G$~-vq%s63h_~dR`}`Oi(>yc!y1F_0w4g$S9Ej&WnL%mRM=nc zDd?AuMR}73%mDYeLSP6iedxFE8QQcCnPF%#uR>1^xQfLqq!)wTIqzW@YRG0j*Wvhb)4%;dlo92;`rSfi>%*mZut`KZF z2m@koImZ2f4S=#XG?O8+?m~MU{M8W>X+)t#5N^(pt`m|V{>0xP4zDklB$2S1d~mt= zsMCR0Tw!qK*RB8C*V3#*iWdV+oxmy2eQBjtT{6hjwpIh|2T(sKaM%?5K&S@fN|4Dl z2Q)~iu(BE7VI<^)L+8?=-2!aQx9f4O5}fJK_YTBw$w;0=9cJSJtP{6->~o*x$G8`X zfOrxO6jxjV+|E}b)ipqO?r(N}xxs2E^{ozKK-Bq~idIXIcU#}!>wsb4Aj^0bM(aA? zgkfGzy_f_rjJInv;Ase;8*Ok%Jpl^?_Vf(b8d1R?NIMaKix73}Ke^VU3SUJ&8`VCz zJF;RCs$L-lc-AVfPLPQL7kFgcJ_!JufZWS7yZ3AQKr8#M%<+rkBHr;SRlXR8Eu!ye z0SO*2hz_D%%;Y_> zJomBtHt=aawF2FpTx_fP#xq10!$E-bhISy;#FjHD+p(N{d+fRdELdt ze|gqEg~>Y78)O&;g!0cPA|Y}VAkRCloTikmt2n0k7q^$rhNo?7{qdNgQHWTh(l=_j zwFD_nQ~CF|(F1|?ojAOehhCn(Rbk^FfpOI%Q^(TFjWn{ok*lb9^m+Fq#<@}CV^Wwt zC$+X?qPOn5hpUdt{iKW|5hnnWn&`2Tzy8&p8rsqZY;)8p2(>AX?|YmNQWSutRtFhD zsj{@Zzr9dceW+uPy8a3%2bnvV0OX2#@eDvCf^fP*9sOBy`U#6W`7VrEMq!NH4G|Wn| zkff>y4MKK;i&)_!4itv&S^?nH3IMx*gt-wH6KS>`=X;~vkRp#bw>W?sg<{x;V5XS! zjPgq!`yv+ru@+DxB&lr%RMfKH4fVIKFxZg@_!Rxk8`*8O+bV1TR9*3_v0C^RnoX{{ zhYge{&u!SPq29`&FH#HRvnyo)?hD!DA|*TAnlB<}DTJHW<3-TXY6r(&{5}9)S8S;+ zUJmpX^nxtBgmN|E#_Tz}-h=Jg4FQk<8uTIpF9@wDfFu6;f&SF5L@AuEHnL;!(-I0L zFpM(dSPxRSw@QjM8isy#A)(MnbODiz?+jG3IkjCcdbp42iP~OV_EvvP@kULX*Ys^utS{mj7C`)wHFLUpB-)_kDo#PuHo6$@`aI|0=Jj$ zhG8z{*B@%AxUNB@q$OYNoAe(+U)n!_He%P`Qb?3#xG|3I8&kb&M=r=`qxl1O;X@N! zT0Qhe6@o!z44`@y|G>m`p_R|?A;d}>X?cgrZNNO*>os#i?*XC!cqjk0lYqYluga2 zGSID84j=Q+R59(R{p={LxK3Mg3n)}(_vc<~7?Is^5&ilLw6th9i(kB&GIgY6cZ|Pd zD@nvO;E#lWUMLbYSE6<<<+Pd*%4$HA=3+7kfB42+Tb; z|A1gEFQqX{r%=R`DaA7?)R`%!M@LpzplRm@?FItAeT3ZvO4BU;4KwwNIvO&hww-QBw}D|K)?*Jmp$={mH=Xl z>Jf^C1bw6UL4ZcEb@r3l4cfzbG=G>l)$no)uUP0P-arsvKhf0E!-)e-8MIp<7@U=O z&ouQguwPYPYJw49Eewh{sIlVxN5Cr(-m~NQQG5_!8cK5Q`$wgeW}%BGKA)EX_tz@8uROcQxdTnu~w?$X}Aen$5c{44aOhE-!?w)UUa;K8#6 z$@e^TSEPd&+fq9rKez=%VwkNf1hm(O`*xJa^c6u803ewFXGRoKA@m6oD%41N@gpHD zam)}f=Eoud6>og^NEBKfO8h4vWbyS@9A@lFjcE6&FOlPC_`}I?&f~6VbK&?>xFNf~ zRvQ}-%6&DQ)wE41SslkSjq8=&s}9W5G;>m6>QZR%1<$QGi(eWR|MY7^8ehE^j^?Sg;0EFT}?a5O5&(+*~rwUJ5i>R()@uHwH80 z943Dn9tI`aI!4uA*RGNZ7axrs-B!ij!HaQ$)hzF6Y zD6*N&X{ZMIhgiz24t83_Y%m;t-z3$KB;b;EGX$Js$L6e>uAB^VJ;FJ#>j6jJcR)J~ zcqq<21^rQfep?+Jx?D8W{`a4sb9BhcN!aAWMDgG4PCM`jNiUvt8J+IliaX*;O;7!7xW2WE>_ zp_}??1E(so^fG@LTA?%6eFg7;Ik$vNctI8n-mFMnWDf`g+ieI^9up76T&I%_N^LVYE!$zgUO*3Ii)ZrUSPSDVo5(wE~eq{Y}3J z6@t>WWeYvMl_A7dzc6o`9aV<}s84VRU~KhmLiRc|Nw9hsu{LquCK*But8B$!%1H@o zMvrQ&`R!dj9fMJe+`haT`P4)I`~duj4UuMuVZ7NZ(qm2-p6~TcGaMh{AU(dv{=WBq zogp%ergHcjx5vL5x;APzd~N@vfe}=P!AxlpCP-uh7yLLaVEaG;_bICHrRSZKpiz-; zmtjM+fD!3_SJR7uJlis;0GldB4STGGZ$_W%-+xM9PE-VJivsx!JE$su7rM=`w877P z=~^<^Q6%L}9_E=I`CdHhaSxM?%O!hj+t3LhZO zpclbr|E@eA_(^L*%iyi%y3OsHE~i0_Pz25shzlhgld)W?iWb2~^8mnwo+c;=^OQO!PqB`&8*f*kGh6(y+K>_hgX%frk5Lh;Q5tqu zLzxBSOd;F!=(xpiCGuMhMqRV+iiKc5M->LHDH!DSQzK+#~CB2hhN66BdjR^l1650Fkrg29B_mIIE0V00Q@qFZ} z<1jh+Nx6Y8U!{d4EhE`;rH_7ZaQgifLoV^AX{tmArHe0en=nbmPmlSR@}4~)HuA5u zCKT4)MD;bLx^fRS_AM6RpWb9D9ETAbyKYFyXKGuKe`aHEKk(}0HsBGM{+GJzi=&$b zQ5~OE1U5Piz&wGY4sf+**Q=H$e*Jq=rY`x6U*o>T@K7^3)kF>$kcs) zP2{K*f{A?t$xT)lTyf(}ni6jfy}0ML)ZK1f&uJJA#f9$*7%U~LC%7z+>|3b-I9->_ zh`z^HK}e7FWP;)XzxZ#U7_~}TUPYVa&ejH=T24E8o3uQ5-uIM_fPl{5zTdMGlE(wK zHooIj;ap)rz1?pR!eKZ9h*Z6dz-wbu5|aLAv1g&M7m_mqF>9S?S;|rfmsmXn)M2^H z*8u6&ePO>gd9#tEGT1x0uBp7v-y;G?4&&KdfWA`};A}~Ek{iQz+r5GBSGW_{g1H^F z%;>b+;q~m&W>g}`VhOx z;-^+MrF{)C`$|tNrBSns0lgGpuVVAEf_O2lb${1Pk3>*zX68=NZp=qf`{Js#&`G}y z=9ZpfMeJYd7^hnc(O*DV5}^_o!g*d(AU=q~?Sj?>o^zPcqZ$!a4TTyzy6-{|YWif< z(Wu2z&|?YT0-fV)rqDSCJRq&(WCZQbVUXFQ118D@)PkqbC!Hq`Ex`-BwXqe#asugF zalGj27#Uh$3im1HcY-ZKL_`V)rRjYjTH~hyIvC__{${J9`=-rEP9ozt(FQRkM>D=# zwNXQNWH4~mz8RHO)MWi1+>n>SG<)xTNDg5zm4&A?nYzruAsxyfK%CtV_WC5o_|juA zIW<3I%=R~T_9VOeO=9?+gnc#R;WnihMke}omxtt1=T}<(F{Y{OlMjR@bY`oMqZ9jW zcXKB+>0Fo&6HYv~ITQ*{RSc2SGQ>V!#ZZ z&PJ{Hi;CS{0TdDntB~TeUisph78A)ds^cvWBXb2J4)Mr0&+>q|QQC{}y8!2uplypu ze{}>lIjC8D8Zb@`Rs90>!qZMzI0t&DTZeqPz9~{Mfhwj~9SA+zrf-!*AI8NZ4twg^>r)3k`%X zS?)wpm+xA_NOtS%!;DQ;&lTi(=ya(z39K*?3W8h`P;Xwzo=H6gOG6(QdeWf$)Vu zxGRc6qse!DI!l7wWmlbsO%My2^x}@mo&y5^J*KIkf%!x2RPZ+PP8a;zN%K^RM)>l6 zK!0d}l|LUqF5Nl#sQk|*+Jn>^tmbiFRNXjHhBv>vRjVyQ1V0u8+JTZj-zFJh(JXuw zr^wEd!Rj3l_4i>_fiPk1MgZt5vQKCqzqpfodWvwqiGK_TKb9Ae8h8`_sF8lV8A() zI5t~TaIx_hAcPH0N9Ovux5sZ#nL^vc;4S%XvWS?RRFp$G$T`QZ$Jw^<BSoS?al-%!@I$U+9t>xL!N zRofh~4_KOR8KPB!pZo3_vShL_iSSq=dp`=ncb5c@Oz{ci+2gPV#-W7lL!EHCfRocMD?xV{i?cy1Qj7!mKo`4MxN*lFJ$0>2tsQ4T zKPHN4wBo$F_Ir+PtnHAz%;=43+D6cE_OR%D)p3{rP!^oLP_!;&;J0GmC;~q6x?O)S zu-yTJXDtClYgh!pv|u1{-3m09cHe>e2SAb}9$tY<6T;IYrU@srjIKgvNe1A<(^f$C zKmyKYl@SmsL~5Ks@Y1uWfv>|Jl#L}di75xsV8<@)xP5rp&+71bp`7;Ilq`~=0Oqj*ApU%#k*lWCy zTkt$3eFVb#+-?SP$}fO7{*WIOWx>+tb03E?u`jaeD7$h)-+|Fn3U`=+$}tlbwtC}B7dYp zE8ck}Egx#hD;APR_Gulrcdk=`6dfQGzLWuC@*v2UuT2C(O|kT_F7tb*fLNd)5fH4D zxgI$Hzy;ibPwiIFu!RI#b8sLlVt>-g)XhlsY zt}V%&to0lAHs)xeU+k!xycCHrD%^&8WqD|=<#TFi13(CFYSj6XnLfW=pbQZmaZwA%jGb$Mj>XA=Jydv(4X_-ZNzc@JO{_1 zKnX_LwjRg(Q27z0$^ZgE@7AbZLV83KHw#F(!56!XkuL{-DMH=nSSp(`kSO-)+a&&q zVu1-*aQU*5pULPRa9v_48Yqc|G+Dv9$iX+Yg>Ksdy9`IYjl2(kH3J~T<~)raUpRa- zcsjwAGUhRT&me4C@UX6}0)iIytcg_UDzZmR^%^t3Abx&dFclg&#?}%sNpx5m(qyXJ zFH=}4<(I(zg8Xwr>G&0*o%WSzi^bTfOmufS(uh*WAH{b%;~V>v=#&{8CRYmc`{ZrN zb2HZSt2$)SWTi+igh}coMFKMtJ=~}fJj8wlgJ+&0>st`TmBKVBo=5cy(`ENm9 zmrPT&0PcI8m)g+Uy$+8hK2ElbQ=n#ZgBrXO^J+Dm4*HofBEmY=MuYQ$YO5y&FN~Se z2v-*SX3ri*#3JFwdT}coX|2)ZeH2)~arlU#QLD1YN?Qw=@jTk#$~`iT2#1o=Gu{HM z)XS2W9NtVb35&&wSas)j#E0qA-Jw9@14(f}2w>Q|wQtAz$z&P%f;tQxFd0C9=XFiU zh-7oHSWuj%P6RMyjpU3=XW!ibQU5)cL1U=Fj7n>u_V^X@N`RrUOF#Pk<_zH7A#EWf zj2U|#IM{zx&#Ej_SR;~E2fwndo)Uv6_)~qwn z3=-{kfol3@&)pL^%>Ty?3cu&SQ3#gT^W}CgjyqrEM4*|=EZr#J9A)nClr979+?zT5 zwgc1!tlRK@rbqkhm_k1i%h$i6^M;3?W^o^Fu6>1c3=kHATL6|k1vIaTsCaTF3Z4QY zO>oQUWk5RuO8GR79n2h8$J^Wj^zXe%bO1Oc@d5tUfc^y$qbOTIySxm@!P9me3--7= zYo?;id@!I{l&vmp^jC73<$_1XZNspcE~HE@09VI9qJ=gyUav}Q8sWr zNR#ga9}?|v5Eg2NzfLdwnZUzHI8T~PPTn2N2qNDFo&Mq*p?-?L3gCse11agaoH?ao zmSEU$3+}m9weh#$q(Q2lEb9mmL7;)=8RWvz!+wRN8RF^&z$rOp(2YA(PGhljudrL# zG3K? zi536E+=^}vKz$RH^QAAKU3y4&8R+=VS3z%f_?x5iR&aHEbP6{s{_`c{O3-Bbel?zk&!m}#hdVYH6pTfK77a&kH?;Bkws zY-ZR|j$f8fj=SZ-Ba!+-GwPauXs5Al4XDnw>I-rCUdlG4aaoK6{;AUz(|)Rl$ON9C zdxSL>5T6cdIA{Hy_}b=n*X)J*M;gDQxtcb<2UyvgX;GmYfTx`MCFoE?j~P0E>33oR zQRz7lAO_Ma5b)2v0DaT!@w#>bbZtw%EM|EB>9hWPaghCZPsp;tQdME!{9Bl{GV{N8 z{>$%aMfRqgGQ9d%Up4Qp=D+-&x+U1GAaFT!@AH3ac0bHtnz@(%AdsQ)Vo+nlIRo*5 zA2Pd)Q1?n`h!Gg9@_qBzk-dL&y?=2f^Q{AzECgC;&TaC)y57Hd((mk8z;XX!5THZw z0Ut7pKFkhYgQ)&b8{t3P2>Rz$E%toc3jZUUAYQvqR^i>F`hiyAy|tTn;y>91;1&PI zK>)Groh#4(Y4zrt|973lm27N<2wXzdtGYiI?S}h zERgUobui6q1TFf}(@o=dG|7HSn&|ODJXE#!oy_n|r1Akq8gik1H{TU|KVQK2bK`da zDlDb%Wp)GO-2!Hm9{O;oxPp1?Y=4v+vjcRwbS&XYX}h6}&?iIRX~u`Hc+|93WPql82HxYo{)?#%PE{)r=E4#?7y7NtMU=s! zOGy-@dzRq&@J9v}&$ojSuSXTGSerp?MrmKYpcW7tY~Hm}3~;{=7j0%D z@wssGFT%HZh9*P*PxVWyuOuR!^GZ92%q~zX`8gmYzM3sT`jEJ8F6th84kSVl-#8oL z*{>osFrLpe{=emYL}X$)w3{+C)A=8!*z{iG|Bs8Quk5JO#-+H9HY3YD2`?z1rL2Y< zE0D$7<}ug#SO}h5h9&o@>M?>C(9k^(i6v;Dq2W zR_{|v@WCm;%<9deKJ#tWpYVoHDZ$^T1fMV?F^K#LGy1=_5tf`0-Jk}Pxl3!9Qx3|= zoniUF3XJ&r5m`E@X(}3gZO=_9=@az&-%Pu%JS4RIaF>F$3oarRV+KNAx zS+hy8Cal3}=w_NlcM~-64H6LMV%`o_NXKcO-0y}*hNSYGMVI{Cvbt*TYr5u<5rdJZ z9ap-!WG8+7Ys1HdpS&W{kCqdQ;8=~~>(mrYFyqZn+tuCc(c5{zgIPYUTD4NjA)sBx zj$*2<$h+*X<^g{c_PnGAXfFCLyK~t;@%)VVSt9{5rx+Ji;Ri?tJ}lPbe6>yc4nIES zw7!6EHpWb4Jf{6Vm_r?2ezft>R<`{Ig2oVS@te3S0fcbB3mn_80yA=PwQi$lGr7l0 zUM297D#ZT5#p=DWs6lgYl$|t zy9vnH+@~kP$s=uG-fT&@Q{D@Az?zw0VhTi8?O!9SC>sY z0|vYqU|+$H8m}UmumAX*cPp;nwZ|PSrjsYqx7}R{n*y{F?6GnuhE-mlHVi9gJNXqu ztH$lN$4s-%-*GoOJ`{0yZ2r{ZU5t;TgC4K4g@G9!Rw%R1g)FFd9kEy@ zK>MLyf0REo2t9>~W0l%;?I83}Y_bSMCpbpKChXgfMt*iAheN}S3$?8Y1wxNaOep!| zjCI_jb{vW>XeZu^*e87)VKBWknK?dr2Y@9qo1l4hr?l;cs7UKawkXuCK6*<>-N>bn z-JPDN8BJ#){?!fZ^=ysGka+rfudKQBEYjDYlGx$@wzW&rx!CNE>6E4V9h?W+nNj+a zU{q&TVP*l;*@a4X2E*;Ve)!nup>r2$a=z3-av}}oP-{gz6i0fhmWIyJv-+dmrj8E^4+_2!wm zPsqr>BO`ymdVlXE%3M!vRQ0^^Ge!e#W`yGKoF5N5HdL5dSie)EnQw0Ht6-#Q;8??= z9t340Och8j^<+`$yOO1UaZ&#EN9T*TR=w{1J3{pDTRI=r0>hdfRh1M#!w^|r$#c7yQ0CQKTy^7ZzIb;DwTh252T;c(k<`vo@ZkV!_{*C zVvPBmp74JJ1wE;p<2|#+K8*Jywszg zx8Gq!9gd?KNwnqfKf1QYWpdL8Cb@rb^6w_NPbzRKR5+E|DDDc_&WrnkJUZ^oLs`_l zmuhZuDKCVLtaD=gSIE!bLjTk0=Ab$7MCixnw-)nS(vGB5H#9hnaJw!(28zg7RdPKy z0}jL46jJ2;!T=vi*&f-V5@~U?{vWf(KM5RX?eBXTW21v#0g6I|hU9@n^xOZ2QbJkd zydDu`oY4^mG86cn1#0<3675`x;AEENZfNA<%R1l26pHm@jKbi@Zj)Xl1Jfwx!DZI^ z{dtz2PhP4Uw+6Xu|3~h{-sJ~D&kW_8DN-_KrGSv{%)rYFuQd3A-6*l>SRAAjo>sGR zt1s*_k5(LZ^TnzUcE)l^^u3G-NI(05f)mKDzw2H{v+bhm%qhduw=@%GXRU%M0=huM ztRZfqZ-=GCO8H49#Bfux5O6|&;87j(lWv$sPD9W{+dN^f9@eLF&@sr~@t2QGbM-rO zYK#Yuo06I%R_aPyYG%~o+q3i-dq8W9f|jQx=88|P#-=oUTrx?Rq;rEglcyT{Q&LMp zaWjns(rErwKX|}H&c500b-U=t)^`QC|GZFJL*MVq`Tvu8bOr_#Q%~>O$xjN>50DbS z1ySdt6`u&>46+DGz42kap-Q5w(+RCAfN+2YO_*M}Gv|HIFnv8wOZ^qd|x)UcD5 zZUnI%C^Gy{*}}g|q&BH((Zhebe>{7nDAWpa^}Ur zc@q|X0`p92)hJByl{>jV9!W9f9g#U5i{Dm#w;Bw654l%N5AFKF!>^!=lLDnK<>*cS`aV9vHK&ENJzcfq zj{)6vS?vN*JZMli^L-2M7`}h8f#RtG#X2xJA-Oq9jKLHNCVQ`G5*j6_j$>sfdv-6I zdHe6Ny_!l7Pwyek>mu)&I~(209Q;GaI2rseEA-W`Vf3eKWG>ELR`Vvj?q~gL#^z1B zyRhuvzIpb&3l=P!Hwy0axBq*0-bKFs&wu|_#ct}%RBsKc!XvYt$2{6PCP$K_26_%@ zx%oeEFA!E>oN8kSbM~}@wjQ#_Sq^VuHMepcE6Y03(1)d~l#=4sed@XQS*%*AcXdnu zkd~%@cI`N;U|VmyFXD7=I|Hc~K2fX9YzbRIqd1BCrO{(z<;P(0RW9GmTv<58q^5I&<2cdB`xTn2AZhX1fKuEdye3R075@rlT3xw%e8!$DId97(E*ruf zapl|~Zoyse&*cK}0;>{>v4EC1U%i5$oYIc2LXO>IM2J`UAM@!VF4(|b5c%e2Egdp@ zstbLy9EMbu#xYL6Y3h){pcp+zZEf=z?N?N@GT|BQwnZ5P^<{l_il{isIY9m-;J<-p{(h~dV;+?twv zQEi{5#LP+^?qum+P-6)Wj)fXD*3sSM;4mt+@HmU@!_T?9&lG<<4d;TSe;4m^s;Ari zvtx0zoy7&>8jhnShujuLTS%~_3cO8mt)oS{7rvpmXu)0hl`hyEry*RccV)l4ZGP7H z`dg)OEeAqWO_YifX8Dtd3vzIB{#k+qnVZ)-X#2!WJz+}S^?L&jAiaImt`#;evI)3s zj`q>EX>ee1i$^Xug%g-Wr1kdI3Dq1eSGD4Gfkpn~6FR|E%?yHug!*YzUqjPIbo4bp zsd)O~(E@Bm3%#SdE?@lsE2a9%&9`?O)=AgBkklK?q{`VpJ6q=y5)0Zxv?O=gxoGkG zMjW{>?>W$m72zoFbDPi9oszlrLd>x@K@}o;H=9WME*h0=P$Gi&r7JjxcA3Z^+fh07 zHo~@tDtTB+R4RpAaHZgT%)w!fT_dwKu@o0!an49wMg8LvWo@`jc*yONtVO36xYa6! zy0w8hE&UJN?syoDKuy6g)jwmaSB8f)2_;L!U`R#+(0o)kWZDNl&?3mYA%-mqC$;(E z?aYdB54$uUl*l<#(2+#()K+ci-8Ugyjl1PMayk4|>dV%(vxh)0)krC92L9;miNjI4 zSV^KSNKjYezoU4cu!L$+*d36_Zo3&06)gkPolyBg&lg>uGALOnJewQXxQ1ErYm`N4 z-4aUZPDkMzl{}OpcjfUKNC5EbSc^kk%$@AwiYQp}s$}WTLtAhQpmNlz8fL{UtmjU< zXqtu0+Q;>hFG`t`O-WDE+84w<383D)JbUGV(?|w$_@IZW3sDK@iJy;ISxTE+ZoIAQ%RzB^R@0F8MvSMPXA%m-3 zNX-KNcB*7U+6Am!^6$tQ3-(KyjzIT9BIsoy_ZldS2JSrEktAyK?vLpaK6NT=NCVw+ z+UZ}8Bu00w=zHThE$6bC(292yNEGT(i-UR<`bo`6$cy}MW@)aNgXFv3XX`C8jRFvo zM&x=wPbIbOhl;kW*=FZ%5F6M>#IJ%~*(h^+bwY;Ug|_oM1bWgugc@}tar=nPUK(RM zBxSu!b7j`#4bbfe;XU(cera46r2E+}(CdDZ5T9Z(6?NnpxU4t}P*;zcu%OdFyG6@7 zWnpr!oQ5oERK1syU?p<((ZMBr4JO@GSL%P6EWLce?B)401S{*udn%|mxds$|qB&gX z#ebBp4i4U;Z-i!bRY^Mz7M^pXPtnd- zjh`lN__-t2qbmol;;lvH);ASy4#$rXU^DX0epc=jX;D7xc4}ksbx|8WX) zq3?B6Jy{CGt^%BkF{N;>>(k;VjoW5!f@1gc{MaH7n*&&sWW#SmEHFdRZX8eoUzV78vRl}?fgy@%( z{2bgH*+q+ej{g(amAdkHOBPh}(4e3x$2o=6Vf=LH^eR8$<)gRsH0``)WL)oN729J| z>8>NDOcAI}?!@cP+}RIbdc{dETY}LtiaKJN>gNI1;P%(|z93R)Qc8A=e6ouy-GwNQqw09Kh9Dryo_t;OR=3Y+Pf|?pDFFXv} z_1T3zWQX{xPzn>O{8pRee5IZZ;g@}#q^12 zD(Yz+G;J-@66T|J@5={YP7+FGu2xFLeuJtqt84bF-=>z1T66^M^E^CR231gTWm-?U zSe4nz>$p{8kt+Int3bva&q^lm(>x3-e!D-{dBsaF>%=!Oh&UgYO`>0Xa@}jRLx~gJ zwhq5MzTrM`BxiE3VfVfJsGBXIwVN#?3ERjRVSD33^6!$%BN)XzHRLf{mR}U0V-{UX zl1aQkW+1mQvUnDxb4vniX5&AL*O(WJzZx-SP>H80N)9k8hgKx~UEzroSqT z(?T~aI)pD|TKBvZIT)#xmqAdaK$Dig7w&1YLu8CA zyz=c=Y3N*Vol>^ok3jb5mm?i5PfyV^I#eePdF@_Y@^DyS5k6I33pF0Z=(DHyqe4nL zi*HT4tV2r@yR>moPnEUnDc%#LGn+i_`t80DnXOCXC;=_rbLzA&P>^ygrf)X<%CW1v zY*@H-E~LHoL~uu1$0Rxod%kSJjt32un%~^xy*US-NYU+ zh$JLfg(|#iep-u;c43A*6JiaTOBK8YRR{4bly50pztY2K!sFq<-1+>a*RXE!l3NG{ zGL;_+3tGNQDLfhJO#VJ}6S;L8s6?~K;Kx}2{cHxyr&KTnLbckH-mktvq13ft4h_NH zUv=6_eGNJ^FSrL(G7FPr+#2&7H;>;f(ltJJ*sU>aT|kt z%0yJqWM8w=c7)G6!~&sD4?V6he)`Qo;N%AOf#RmKN&=Z((i+}{WgWQ_k&80?Fw=QB zirz_tB&ivOJ#ZM7qMZNxmOI=8r zB1OxoKkjuy|3cn#A`*6d4`xrvo@Wip;(Wvz+rp|@yd;KPWIQ&tk-dWOM7wffgvH?n z5f%%UExNXT#=3()|2@I^^sAp%;D5XVUg^owZ?O?EK=M+s>`6shq{Z8h1b;bf17hiU zKR*Jc`@DZH_+S0^D-kVT4#GRuEg4KWwP*P9ngu^>p8fjgNwYvm%xsvwE`0XF?SavR xett27&G_c9gG%E6Snxdv-=Lol_>jGn&qjL!qmB_sY0x_k?z7nYVE2y~{|`uFp-un* literal 0 HcmV?d00001 diff --git a/docs/web-ui.md b/docs/web-ui.md index 3c35dbeec86a2..e2e612cef3e54 100644 --- a/docs/web-ui.md +++ b/docs/web-ui.md @@ -407,6 +407,34 @@ Here is the list of SQL metrics: +## Structured Streaming Tab +When running Structured Streaming jobs in micro-batch mode, a Structured Streaming tab will be +available on the Web UI. The overview page displays some brief statistics for running and completed +queries. Also, you can check the latest exception of a failed query. For detailed statistics, please +click a "run id" in the tables. + +

+ Structured Streaming Query Statistics +

+ +The statistics page displays some useful metrics for insight into the status of your streaming +queries. Currently, it contains the following metrics. + +* **Input Rate.** The aggregate (across all sources) rate of data arriving. +* **Process Rate.** The aggregate (across all sources) rate at which Spark is processing data. +* **Input Rows.** The aggregate (across all sources) number of records processed in a trigger. +* **Batch Duration.** The process duration of each batch. +* **Operation Duration.** The amount of time taken to perform various operations in milliseconds. +The tracked operations are listed as follows. + * addBatch: Adds result data of the current batch to the sink. + * getBatch: Gets a new batch of data to process. + * latestOffset: Gets the latest offsets for sources. + * queryPlanning: Generates the execution plan. + * walCommit: Writes the offsets to the metadata log. + +As an early-release version, the statistics page is still under development and will be improved in +future releases. + ## Streaming Tab The web UI includes a Streaming tab if the application uses Spark streaming. This tab displays scheduling delay and processing time for each micro-batch in the data stream, which can be useful From b44acee95343f3508f12384b17e43cde65220924 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Tue, 26 May 2020 14:40:58 +0900 Subject: [PATCH 10/14] [SPARK-31673][SQL] QueryExection.debug.toFile() to take an addtional explain mode param ### What changes were proposed in this pull request? Currently QueryExecution.debug.toFile dumps the query plan information in a fixed format. This PR adds an additional explain mode parameter that writes the debug information as per the user supplied format. ``` df.queryExecution.debug.toFile("/tmp/plan.txt", explainMode = ExplainMode.fromString("formatted")) ``` ``` == Physical Plan == * Filter (2) +- Scan hive default.s1 (1) (1) Scan hive default.s1 Output [2]: [c1#15, c2#16] Arguments: [c1#15, c2#16], HiveTableRelation `default`.`s1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [c1#15, c2#16] (2) Filter [codegen id : 1] Input [2]: [c1#15, c2#16] Condition : (isnotnull(c1#15) AND (c1#15 > 0)) == Whole Stage Codegen == Found 1 WholeStageCodegen subtrees. == Subtree 1 / 1 (maxMethodCodeSize:220; maxConstantPoolSize:105(0.16% used); numInnerClasses:0) == *(1) Filter (isnotnull(c1#15) AND (c1#15 > 0)) +- Scan hive default.s1 [c1#15, c2#16], HiveTableRelation `default`.`s1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [c1#15, c2#16] Generated code: /* 001 */ public Object generate(Object[] references) { /* 002 */ return new GeneratedIteratorForCodegenStage1(references); /* 003 */ } /* 004 */ /* 005 */ // codegenStageId=1 /* 006 */ final class GeneratedIteratorForCodegenStage1 extends org.apache.spark.sql.execution.BufferedRowIterator { /* 007 */ private Object[] references; /* 008 */ private scala.collection.Iterator[] inputs; /* 009 */ private scala.collection.Iterator inputadapter_input_0; /* 010 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] filter_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1]; /* 011 */ /* 012 */ public GeneratedIteratorForCodegenStage1(Object[] references) { /* 013 */ this.references = references; /* 014 */ } /* 015 */ /* 016 */ public void init(int index, scala.collection.Iterator[] inputs) { /* 017 */ partitionIndex = index; /* 018 */ this.inputs = inputs; /* 019 */ inputadapter_input_0 = inputs[0]; /* 020 */ filter_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0); /* 021 */ /* 022 */ } /* 023 */ /* 024 */ protected void processNext() throws java.io.IOException { /* 025 */ while ( inputadapter_input_0.hasNext()) { /* 026 */ InternalRow inputadapter_row_0 = (InternalRow) inputadapter_input_0.next(); /* 027 */ /* 028 */ do { /* 029 */ boolean inputadapter_isNull_0 = inputadapter_row_0.isNullAt(0); /* 030 */ int inputadapter_value_0 = inputadapter_isNull_0 ? /* 031 */ -1 : (inputadapter_row_0.getInt(0)); /* 032 */ /* 033 */ boolean filter_value_2 = !inputadapter_isNull_0; /* 034 */ if (!filter_value_2) continue; /* 035 */ /* 036 */ boolean filter_value_3 = false; /* 037 */ filter_value_3 = inputadapter_value_0 > 0; /* 038 */ if (!filter_value_3) continue; /* 039 */ /* 040 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1); /* 041 */ /* 042 */ boolean inputadapter_isNull_1 = inputadapter_row_0.isNullAt(1); /* 043 */ int inputadapter_value_1 = inputadapter_isNull_1 ? /* 044 */ -1 : (inputadapter_row_0.getInt(1)); /* 045 */ filter_mutableStateArray_0[0].reset(); /* 046 */ /* 047 */ filter_mutableStateArray_0[0].zeroOutNullBytes(); /* 048 */ /* 049 */ filter_mutableStateArray_0[0].write(0, inputadapter_value_0); /* 050 */ /* 051 */ if (inputadapter_isNull_1) { /* 052 */ filter_mutableStateArray_0[0].setNullAt(1); /* 053 */ } else { /* 054 */ filter_mutableStateArray_0[0].write(1, inputadapter_value_1); /* 055 */ } /* 056 */ append((filter_mutableStateArray_0[0].getRow())); /* 057 */ /* 058 */ } while(false); /* 059 */ if (shouldStop()) return; /* 060 */ } /* 061 */ } /* 062 */ /* 063 */ } ``` ### Why are the changes needed? Hopefully enhances the usability of debug.toFile(..) ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added a test in QueryExecutionSuite Closes #28493 from dilipbiswal/write_to_file. Authored-by: Dilip Biswal Signed-off-by: HyukjinKwon --- .../spark/sql/execution/QueryExecution.scala | 104 ++++++++++++------ .../sql/execution/QueryExecutionSuite.scala | 20 ++++ 2 files changed, 90 insertions(+), 34 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 1df812d1aa809..89915d254883d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -22,6 +22,7 @@ import java.util.UUID import org.apache.hadoop.fs.Path +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.{InternalRow, QueryPlanningTracker} @@ -50,7 +51,7 @@ import org.apache.spark.util.Utils class QueryExecution( val sparkSession: SparkSession, val logical: LogicalPlan, - val tracker: QueryPlanningTracker = new QueryPlanningTracker) { + val tracker: QueryPlanningTracker = new QueryPlanningTracker) extends Logging { // TODO: Move the planner an optimizer into here from SessionState. protected def planner = sparkSession.sessionState.planner @@ -133,26 +134,42 @@ class QueryExecution( tracker.measurePhase(phase)(block) } - def simpleString: String = simpleString(false) - - def simpleString(formatted: Boolean): String = withRedaction { + def simpleString: String = { val concat = new PlanStringConcat() - concat.append("== Physical Plan ==\n") + simpleString(false, SQLConf.get.maxToStringFields, concat.append) + withRedaction { + concat.toString + } + } + + private def simpleString( + formatted: Boolean, + maxFields: Int, + append: String => Unit): Unit = { + append("== Physical Plan ==\n") if (formatted) { try { - ExplainUtils.processPlan(executedPlan, concat.append) + ExplainUtils.processPlan(executedPlan, append) } catch { - case e: AnalysisException => concat.append(e.toString) - case e: IllegalArgumentException => concat.append(e.toString) + case e: AnalysisException => append(e.toString) + case e: IllegalArgumentException => append(e.toString) } } else { - QueryPlan.append(executedPlan, concat.append, verbose = false, addSuffix = false) + QueryPlan.append(executedPlan, + append, verbose = false, addSuffix = false, maxFields = maxFields) } - concat.append("\n") - concat.toString + append("\n") } def explainString(mode: ExplainMode): String = { + val concat = new PlanStringConcat() + explainString(mode, SQLConf.get.maxToStringFields, concat.append) + withRedaction { + concat.toString + } + } + + private def explainString(mode: ExplainMode, maxFields: Int, append: String => Unit): Unit = { val queryExecution = if (logical.isStreaming) { // This is used only by explaining `Dataset/DataFrame` created by `spark.readStream`, so the // output mode does not matter since there is no `Sink`. @@ -165,19 +182,19 @@ class QueryExecution( mode match { case SimpleMode => - queryExecution.simpleString + queryExecution.simpleString(false, maxFields, append) case ExtendedMode => - queryExecution.toString + queryExecution.toString(maxFields, append) case CodegenMode => try { - org.apache.spark.sql.execution.debug.codegenString(queryExecution.executedPlan) + org.apache.spark.sql.execution.debug.writeCodegen(append, queryExecution.executedPlan) } catch { - case e: AnalysisException => e.toString + case e: AnalysisException => append(e.toString) } case CostMode => - queryExecution.stringWithStats + queryExecution.stringWithStats(maxFields, append) case FormattedMode => - queryExecution.simpleString(formatted = true) + queryExecution.simpleString(formatted = true, maxFields = maxFields, append) } } @@ -204,27 +221,39 @@ class QueryExecution( override def toString: String = withRedaction { val concat = new PlanStringConcat() - writePlans(concat.append, SQLConf.get.maxToStringFields) - concat.toString + toString(SQLConf.get.maxToStringFields, concat.append) + withRedaction { + concat.toString + } + } + + private def toString(maxFields: Int, append: String => Unit): Unit = { + writePlans(append, maxFields) } - def stringWithStats: String = withRedaction { + def stringWithStats: String = { val concat = new PlanStringConcat() + stringWithStats(SQLConf.get.maxToStringFields, concat.append) + withRedaction { + concat.toString + } + } + + private def stringWithStats(maxFields: Int, append: String => Unit): Unit = { val maxFields = SQLConf.get.maxToStringFields // trigger to compute stats for logical plans try { optimizedPlan.stats } catch { - case e: AnalysisException => concat.append(e.toString + "\n") + case e: AnalysisException => append(e.toString + "\n") } // only show optimized logical plan and physical plan - concat.append("== Optimized Logical Plan ==\n") - QueryPlan.append(optimizedPlan, concat.append, verbose = true, addSuffix = true, maxFields) - concat.append("\n== Physical Plan ==\n") - QueryPlan.append(executedPlan, concat.append, verbose = true, addSuffix = false, maxFields) - concat.append("\n") - concat.toString + append("== Optimized Logical Plan ==\n") + QueryPlan.append(optimizedPlan, append, verbose = true, addSuffix = true, maxFields) + append("\n== Physical Plan ==\n") + QueryPlan.append(executedPlan, append, verbose = true, addSuffix = false, maxFields) + append("\n") } /** @@ -261,19 +290,26 @@ class QueryExecution( /** * Dumps debug information about query execution into the specified file. * + * @param path path of the file the debug info is written to. * @param maxFields maximum number of fields converted to string representation. + * @param explainMode the explain mode to be used to generate the string + * representation of the plan. */ - def toFile(path: String, maxFields: Int = Int.MaxValue): Unit = { + def toFile( + path: String, + maxFields: Int = Int.MaxValue, + explainMode: Option[String] = None): Unit = { val filePath = new Path(path) val fs = filePath.getFileSystem(sparkSession.sessionState.newHadoopConf()) val writer = new BufferedWriter(new OutputStreamWriter(fs.create(filePath))) - val append = (s: String) => { - writer.write(s) - } try { - writePlans(append, maxFields) - writer.write("\n== Whole Stage Codegen ==\n") - org.apache.spark.sql.execution.debug.writeCodegen(writer.write, executedPlan) + val mode = explainMode.map(ExplainMode.fromString(_)).getOrElse(ExtendedMode) + explainString(mode, maxFields, writer.write) + if (mode != CodegenMode) { + writer.write("\n== Whole Stage Codegen ==\n") + org.apache.spark.sql.execution.debug.writeCodegen(writer.write, executedPlan) + } + log.info(s"Debug information was written at: $filePath") } finally { writer.close() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index eca39f3f81726..5c35cedba9bab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -53,6 +53,7 @@ class QueryExecutionSuite extends SharedSparkSession { s"*(1) Range (0, $expected, step=1, splits=2)", "")) } + test("dumping query execution info to a file") { withTempDir { dir => val path = dir.getCanonicalPath + "/plans.txt" @@ -93,6 +94,25 @@ class QueryExecutionSuite extends SharedSparkSession { assert(exception.getMessage.contains("Illegal character in scheme name")) } + test("dumping query execution info to a file - explainMode=formatted") { + withTempDir { dir => + val path = dir.getCanonicalPath + "/plans.txt" + val df = spark.range(0, 10) + df.queryExecution.debug.toFile(path, explainMode = Option("formatted")) + assert(Source.fromFile(path).getLines.toList + .takeWhile(_ != "== Whole Stage Codegen ==").map(_.replaceAll("#\\d+", "#x")) == List( + "== Physical Plan ==", + s"* Range (1)", + "", + "", + s"(1) Range [codegen id : 1]", + "Output [1]: [id#xL]", + s"Arguments: Range (0, 10, step=1, splits=Some(2))", + "", + "")) + } + } + test("limit number of fields by sql config") { def relationPlans: String = { val ds = spark.createDataset(Seq(QueryExecutionTestRecord( From 87d34e6b969e9ceba207b996506156bd36f3dd64 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 26 May 2020 12:13:28 +0000 Subject: [PATCH 11/14] [SPARK-31820][SQL][TESTS] Fix flaky JavaBeanDeserializationSuite ### What changes were proposed in this pull request? Modified formatting of expected timestamp strings in the test `JavaBeanDeserializationSuite`.`testSpark22000` to correctly format timestamps with **zero** seconds fraction. Current implementation outputs `.0` but must be empty string. From SPARK-31820 failure: - should be `2020-05-25 12:39:17` - but incorrect expected string is `2020-05-25 12:39:17.0` ### Why are the changes needed? To make `JavaBeanDeserializationSuite` stable, and avoid test failures like https://github.com/apache/spark/pull/28630#issuecomment-633695723 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? I changed https://github.com/apache/spark/blob/7dff3b125de23a4d6ce834217ee08973b259414c/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java#L207 to ```java new java.sql.Timestamp((System.currentTimeMillis() / 1000) * 1000), ``` to force zero seconds fraction. Closes #28639 from MaxGekk/fix-JavaBeanDeserializationSuite. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../spark/sql/JavaBeanDeserializationSuite.java | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java index 5603cb988b8e7..af0a22b036030 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java @@ -18,6 +18,8 @@ package test.org.apache.spark.sql; import java.io.Serializable; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; import java.time.Instant; import java.time.LocalDate; import java.util.*; @@ -210,6 +212,17 @@ private static Row createRecordSpark22000Row(Long index) { return new GenericRow(values); } + private static String timestampToString(Timestamp ts) { + String timestampString = String.valueOf(ts); + String formatted = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(ts); + + if (timestampString.length() > 19 && !timestampString.substring(19).equals(".0")) { + return formatted + timestampString.substring(19); + } else { + return formatted; + } + } + private static RecordSpark22000 createRecordSpark22000(Row recordRow) { RecordSpark22000 record = new RecordSpark22000(); record.setShortField(String.valueOf(recordRow.getShort(0))); @@ -219,7 +232,7 @@ private static RecordSpark22000 createRecordSpark22000(Row recordRow) { record.setDoubleField(String.valueOf(recordRow.getDouble(4))); record.setStringField(recordRow.getString(5)); record.setBooleanField(String.valueOf(recordRow.getBoolean(6))); - record.setTimestampField(String.valueOf(recordRow.getTimestamp(7))); + record.setTimestampField(timestampToString(recordRow.getTimestamp(7))); // This would figure out that null value will not become "null". record.setNullIntField(null); return record; From 7fb2275f009c8744560c3247decdc106a8bca86f Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 27 May 2020 10:15:33 +0900 Subject: [PATCH 12/14] Revert "[SPARK-31788][CORE][PYTHON] Fix UnionRDD of PairRDDs" This reverts commit a61911c50c391e61038cf01611629d2186d17a76. --- python/pyspark/context.py | 12 ++---------- python/pyspark/tests/test_rdd.py | 9 --------- 2 files changed, 2 insertions(+), 19 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index b80149afa2af4..4f29f2f0be1e8 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -25,7 +25,6 @@ from tempfile import NamedTemporaryFile from py4j.protocol import Py4JError -from py4j.java_gateway import is_instance_of from pyspark import accumulators from pyspark.accumulators import Accumulator @@ -865,17 +864,10 @@ def union(self, rdds): first_jrdd_deserializer = rdds[0]._jrdd_deserializer if any(x._jrdd_deserializer != first_jrdd_deserializer for x in rdds): rdds = [x._reserialize() for x in rdds] - gw = SparkContext._gateway cls = SparkContext._jvm.org.apache.spark.api.java.JavaRDD - is_jrdd = is_instance_of(gw, rdds[0]._jrdd, cls) - jrdds = gw.new_array(cls, len(rdds)) + jrdds = SparkContext._gateway.new_array(cls, len(rdds)) for i in range(0, len(rdds)): - if is_jrdd: - jrdds[i] = rdds[i]._jrdd - else: - # zip could return JavaPairRDD hence we ensure `_jrdd` - # to be `JavaRDD` by wrapping it in a `map` - jrdds[i] = rdds[i].map(lambda x: x)._jrdd + jrdds[i] = rdds[i]._jrdd return RDD(self._jsc.union(jrdds), self, rdds[0]._jrdd_deserializer) def broadcast(self, value): diff --git a/python/pyspark/tests/test_rdd.py b/python/pyspark/tests/test_rdd.py index 04dfe68e57a3a..62ad4221d7078 100644 --- a/python/pyspark/tests/test_rdd.py +++ b/python/pyspark/tests/test_rdd.py @@ -168,15 +168,6 @@ def test_zip_chaining(self): set([(x, (x, x)) for x in 'abc']) ) - def test_union_pair_rdd(self): - # Regression test for SPARK-31788 - rdd = self.sc.parallelize([1, 2]) - pair_rdd = rdd.zip(rdd) - self.assertEqual( - self.sc.union([pair_rdd, pair_rdd]).collect(), - [((1, 1), (2, 2)), ((1, 1), (2, 2))] - ) - def test_deleting_input_files(self): # Regression test for SPARK-1025 tempFile = tempfile.NamedTemporaryFile(delete=False) From 8f1d77488c07a50b90a113f7faacd3d96f25646b Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Tue, 26 May 2020 18:21:22 -0700 Subject: [PATCH 13/14] [SPARK-31821][BUILD] Remove mssql-jdbc dependencies from Hadoop 3.2 profile ### What changes were proposed in this pull request? There is an unnecessary dependency for `mssql-jdbc`. In this PR I've removed it. ### Why are the changes needed? Unnecessary dependency. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the Jenkins with the following configuration. - [x] Pass the dependency test. - [x] SBT with Hadoop-3.2 (https://github.com/apache/spark/pull/28640#issuecomment-634192512) - [ ] Maven with Hadoop-3.2 Closes #28640 from gaborgsomogyi/SPARK-31821. Authored-by: Gabor Somogyi Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 1 - pom.xml | 4 ++++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 3c3ce2dcdd6d4..b5a10b5dba378 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -183,7 +183,6 @@ metrics-jmx/4.1.1//metrics-jmx-4.1.1.jar metrics-json/4.1.1//metrics-json-4.1.1.jar metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar minlog/1.3.0//minlog-1.3.0.jar -mssql-jdbc/6.2.1.jre7//mssql-jdbc-6.2.1.jre7.jar netty-all/4.1.47.Final//netty-all-4.1.47.Final.jar nimbus-jose-jwt/4.41.1//nimbus-jose-jwt-4.41.1.jar objenesis/2.5.1//objenesis-2.5.1.jar diff --git a/pom.xml b/pom.xml index fd4cebcd37319..deaf87f15539c 100644 --- a/pom.xml +++ b/pom.xml @@ -1357,6 +1357,10 @@ com.zaxxer HikariCP-java7 + + com.microsoft.sqlserver + mssql-jdbc + From 8f2b6f3a0b9d4297cb1f62e682958239fd6f9dbd Mon Sep 17 00:00:00 2001 From: beliefer Date: Wed, 27 May 2020 15:05:06 +0900 Subject: [PATCH 14/14] [SPARK-31393][SQL][FOLLOW-UP] Show the correct alias in schema for expression ### What changes were proposed in this pull request? Some alias of expression can not display correctly in schema. This PR will fix them. - `ln` - `rint` - `lcase` - `position` ### Why are the changes needed? Improve the implement of some expression. ### Does this PR introduce _any_ user-facing change? 'Yes'. This PR will let user see the correct alias in schema. ### How was this patch tested? Jenkins test. Closes #28551 from beliefer/show-correct-alias-in-schema. Lead-authored-by: beliefer Co-authored-by: gengjiaan Signed-off-by: HyukjinKwon --- .../sql/catalyst/analysis/FunctionRegistry.scala | 4 ++-- .../sql/catalyst/expressions/mathExpressions.scala | 5 ++++- .../catalyst/expressions/stringExpressions.scala | 6 +++++- .../sql-functions/sql-expression-schema.md | 8 ++++---- .../results/ansi/string-functions.sql.out | 2 +- .../sql-tests/results/postgreSQL/numeric.sql.out | 14 +++++++------- .../sql-tests/results/string-functions.sql.out | 2 +- 7 files changed, 24 insertions(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index d4799cace4531..e2559d4c07297 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -339,7 +339,7 @@ object FunctionRegistry { expression[GetJsonObject]("get_json_object"), expression[InitCap]("initcap"), expression[StringInstr]("instr"), - expression[Lower]("lcase"), + expression[Lower]("lcase", true), expression[Length]("length"), expression[Levenshtein]("levenshtein"), expression[Like]("like"), @@ -350,7 +350,7 @@ object FunctionRegistry { expression[StringTrimLeft]("ltrim"), expression[JsonTuple]("json_tuple"), expression[ParseUrl]("parse_url"), - expression[StringLocate]("position"), + expression[StringLocate]("position", true), expression[FormatString]("printf", true), expression[RegExpExtract]("regexp_extract"), expression[RegExpReplace]("regexp_replace"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index 66e6334e3a450..8c6fbc0fc8e44 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -491,7 +491,9 @@ case class Factorial(child: Expression) extends UnaryExpression with ImplicitCas > SELECT _FUNC_(1); 0.0 """) -case class Log(child: Expression) extends UnaryLogExpression(StrictMath.log, "LOG") +case class Log(child: Expression) extends UnaryLogExpression(StrictMath.log, "LOG") { + override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("ln") +} @ExpressionDescription( usage = "_FUNC_(expr) - Returns the logarithm of `expr` with base 2.", @@ -546,6 +548,7 @@ case class Log1p(child: Expression) extends UnaryLogExpression(StrictMath.log1p, // scalastyle:on line.size.limit case class Rint(child: Expression) extends UnaryMathExpression(math.rint, "ROUND") { override def funcName: String = "rint" + override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("rint") } @ExpressionDescription( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index 0b9fb8f85fe3c..876588e096d4a 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -365,6 +365,9 @@ case class Lower(child: Expression) extends UnaryExpression with String2StringEx override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { defineCodeGen(ctx, ev, c => s"($c).toLowerCase()") } + + override def prettyName: String = + getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("lower") } /** A base trait for functions that compare two strings, returning a boolean. */ @@ -1182,7 +1185,8 @@ case class StringLocate(substr: Expression, str: Expression, start: Expression) """) } - override def prettyName: String = "locate" + override def prettyName: String = + getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("locate") } /** diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index 8949b62f0a512..d245aa5a17345 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -158,12 +158,12 @@ | org.apache.spark.sql.catalyst.expressions.LessThanOrEqual | <= | SELECT 2 <= 2 | struct<(2 <= 2):boolean> | | org.apache.spark.sql.catalyst.expressions.Levenshtein | levenshtein | SELECT levenshtein('kitten', 'sitting') | struct | | org.apache.spark.sql.catalyst.expressions.Like | like | SELECT like('Spark', '_park') | struct | -| org.apache.spark.sql.catalyst.expressions.Log | ln | SELECT ln(1) | struct | +| org.apache.spark.sql.catalyst.expressions.Log | ln | SELECT ln(1) | struct | | org.apache.spark.sql.catalyst.expressions.Log10 | log10 | SELECT log10(10) | struct | | org.apache.spark.sql.catalyst.expressions.Log1p | log1p | SELECT log1p(0) | struct | | org.apache.spark.sql.catalyst.expressions.Log2 | log2 | SELECT log2(2) | struct | | org.apache.spark.sql.catalyst.expressions.Logarithm | log | SELECT log(10, 100) | struct | -| org.apache.spark.sql.catalyst.expressions.Lower | lcase | SELECT lcase('SparkSql') | struct | +| org.apache.spark.sql.catalyst.expressions.Lower | lcase | SELECT lcase('SparkSql') | struct | | org.apache.spark.sql.catalyst.expressions.Lower | lower | SELECT lower('SparkSql') | struct | | org.apache.spark.sql.catalyst.expressions.MakeDate | make_date | SELECT make_date(2013, 7, 15) | struct | | org.apache.spark.sql.catalyst.expressions.MakeInterval | make_interval | SELECT make_interval(100, 11, 1, 1, 12, 30, 01.001001) | struct | @@ -219,7 +219,7 @@ | org.apache.spark.sql.catalyst.expressions.Remainder | mod | SELECT 2 % 1.8 | struct<(CAST(CAST(2 AS DECIMAL(1,0)) AS DECIMAL(2,1)) % CAST(1.8 AS DECIMAL(2,1))):decimal(2,1)> | | org.apache.spark.sql.catalyst.expressions.Reverse | reverse | SELECT reverse('Spark SQL') | struct | | org.apache.spark.sql.catalyst.expressions.Right | right | SELECT right('Spark SQL', 3) | struct | -| org.apache.spark.sql.catalyst.expressions.Rint | rint | SELECT rint(12.3456) | struct | +| org.apache.spark.sql.catalyst.expressions.Rint | rint | SELECT rint(12.3456) | struct | | org.apache.spark.sql.catalyst.expressions.Rollup | rollup | SELECT name, age, count(*) FROM VALUES (2, 'Alice'), (5, 'Bob') people(age, name) GROUP BY rollup(name, age) | struct | | org.apache.spark.sql.catalyst.expressions.Round | round | SELECT round(2.5, 0) | struct | | org.apache.spark.sql.catalyst.expressions.RowNumber | row_number | N/A | N/A | @@ -251,7 +251,7 @@ | org.apache.spark.sql.catalyst.expressions.Stack | stack | SELECT stack(2, 1, 2, 3) | struct | | org.apache.spark.sql.catalyst.expressions.StringInstr | instr | SELECT instr('SparkSQL', 'SQL') | struct | | org.apache.spark.sql.catalyst.expressions.StringLPad | lpad | SELECT lpad('hi', 5, '??') | struct | -| org.apache.spark.sql.catalyst.expressions.StringLocate | position | SELECT position('bar', 'foobarbar') | struct | +| org.apache.spark.sql.catalyst.expressions.StringLocate | position | SELECT position('bar', 'foobarbar') | struct | | org.apache.spark.sql.catalyst.expressions.StringLocate | locate | SELECT locate('bar', 'foobarbar') | struct | | org.apache.spark.sql.catalyst.expressions.StringRPad | rpad | SELECT rpad('hi', 5, '??') | struct | | org.apache.spark.sql.catalyst.expressions.StringRepeat | repeat | SELECT repeat('123', 2) | struct | diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out index b507713a73d1f..d5c0acb40bb1e 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out @@ -55,7 +55,7 @@ struct -- !query select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aaads', null) -- !query schema -struct +struct -- !query output 4 NULL NULL diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out index e59b9d5b63a40..7b7aeb4ec7934 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out @@ -4654,7 +4654,7 @@ struct -- !query select ln(1.2345678e-28) -- !query schema -struct +struct -- !query output -64.26166165451762 @@ -4662,7 +4662,7 @@ struct -- !query select ln(0.0456789) -- !query schema -struct +struct -- !query output -3.0861187944847437 @@ -4670,7 +4670,7 @@ struct -- !query select ln(0.99949452) -- !query schema -struct +struct -- !query output -5.056077980832118E-4 @@ -4678,7 +4678,7 @@ struct -- !query select ln(1.00049687395) -- !query schema -struct +struct -- !query output 4.967505490136803E-4 @@ -4686,7 +4686,7 @@ struct -- !query select ln(1234.567890123456789) -- !query schema -struct +struct -- !query output 7.11847630129779 @@ -4694,7 +4694,7 @@ struct -- !query select ln(5.80397490724e5) -- !query schema -struct +struct -- !query output 13.271468476626518 @@ -4702,7 +4702,7 @@ struct -- !query select ln(9.342536355e34) -- !query schema -struct +struct -- !query output 80.52247093552418 diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out index 0d37c0d02e61f..20c31b140b009 100644 --- a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out @@ -55,7 +55,7 @@ struct -- !query select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aaads', null) -- !query schema -struct +struct -- !query output 4 NULL NULL